From d729fcba4cd06d348d9714b0af01962423f47dd3 Mon Sep 17 00:00:00 2001 From: monster <60029759+MonsterChenzhuo@users.noreply.github.com> Date: Mon, 17 Jul 2023 17:49:47 +0800 Subject: [PATCH 01/43] [Feature][connector-v2][mongodbcdc]Support source mongodb cdc (#4923) --- LICENSE | 3 +- config/plugin_config | 1 + docs/en/connector-v2/source/MongoDB-CDC.md | 282 +++++++++ plugin-mapping.properties | 1 + .../IncrementalSourceStreamFetcher.java | 14 +- .../connector-cdc-mongodb/pom.xml | 87 +++ .../cdc/mongodb/MongodbIncrementalSource.java | 132 +++++ .../MongodbIncrementalSourceFactory.java | 104 ++++ .../mongodb/config/MongodbSourceConfig.java | 122 ++++ .../config/MongodbSourceConfigProvider.java | 175 ++++++ .../mongodb/config/MongodbSourceOptions.java | 255 ++++++++ .../exception/MongodbConnectorException.java | 28 + .../internal/MongodbClientProvider.java | 45 ++ ...MongoDBConnectorDeserializationSchema.java | 553 ++++++++++++++++++ .../mongodb/sender/SerializableFunction.java | 24 + .../source/dialect/MongodbDialect.java | 146 +++++ .../source/fetch/MongodbFetchTaskContext.java | 210 +++++++ .../source/fetch/MongodbScanFetchTask.java | 253 ++++++++ .../source/fetch/MongodbStreamFetchTask.java | 359 ++++++++++++ .../source/offset/ChangeStreamDescriptor.java | 73 +++ .../source/offset/ChangeStreamOffset.java | 94 +++ .../offset/ChangeStreamOffsetFactory.java | 62 ++ .../splitters/MongodbChunkSplitter.java | 45 ++ .../splitters/SampleBucketSplitStrategy.java | 139 +++++ .../splitters/ShardedSplitStrategy.java | 108 ++++ .../source/splitters/SingleSplitStrategy.java | 54 ++ .../source/splitters/SplitContext.java | 93 +++ .../source/splitters/SplitStrategy.java | 57 ++ .../splitters/SplitVectorSplitStrategy.java | 119 ++++ .../cdc/mongodb/utils/BsonUtils.java | 325 ++++++++++ .../cdc/mongodb/utils/ChunkUtils.java | 45 ++ .../utils/CollectionDiscoveryUtils.java | 171 ++++++ .../cdc/mongodb/utils/MongodbRecordUtils.java | 165 ++++++ .../cdc/mongodb/utils/MongodbUtils.java | 407 +++++++++++++ .../cdc/mongodb/utils/ResumeToken.java | 80 +++ .../MongodbIncrementalSourceFactoryTest.java | 30 + .../cdc/mysql/testutils/UniqueDatabase.java | 10 +- seatunnel-connectors-v2/connector-cdc/pom.xml | 1 + seatunnel-dist/pom.xml | 6 + .../connector-cdc-mongodb-e2e/pom.xml | 71 +++ .../test/java/mongodb/MongoDBContainer.java | 240 ++++++++ .../src/test/java/mongodb/MongodbCDCIT.java | 266 +++++++++ .../src/test/resources/ddl/inventory.js | 24 + .../src/test/resources/ddl/inventoryDDL.js | 32 + .../src/test/resources/ddl/mongodb_cdc.sql | 32 + .../test/resources/docker/mongodb/random.key | 34 ++ .../test/resources/docker/mongodb/setup.js | 39 ++ .../src/test/resources/log4j2-test.properties | 29 + .../test/resources/mongodbcdc_to_mysql.conf | 59 ++ .../seatunnel-connector-v2-e2e/pom.xml | 1 + 50 files changed, 5696 insertions(+), 9 deletions(-) create mode 100644 docs/en/connector-v2/source/MongoDB-CDC.md create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/pom.xml create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSource.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfig.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfigProvider.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/exception/MongodbConnectorException.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/internal/MongodbClientProvider.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/MongoDBConnectorDeserializationSchema.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/SerializableFunction.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbFetchTaskContext.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbScanFetchTask.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbStreamFetchTask.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamDescriptor.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffset.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffsetFactory.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/MongodbChunkSplitter.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SampleBucketSplitStrategy.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/ShardedSplitStrategy.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SingleSplitStrategy.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitContext.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitStrategy.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitVectorSplitStrategy.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/BsonUtils.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ChunkUtils.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/CollectionDiscoveryUtils.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbUtils.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/test/java/mongodb/source/MongodbIncrementalSourceFactoryTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventory.js create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventoryDDL.js create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/mongodb_cdc.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/random.key create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/setup.js create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/log4j2-test.properties create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf diff --git a/LICENSE b/LICENSE index bd06a03806b3..adabba50de63 100644 --- a/LICENSE +++ b/LICENSE @@ -219,6 +219,7 @@ seatunnel-connectors-v2/connector-cdc/connector-base/src/main/java/org/apache/se seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql from https://github.com/ververica/flink-cdc-connectors seatunnel-connectors-v2/connector-cdc/connector-base/src/main/java/org/apache/seatunnel/connectors/cdc/debezium from https://github.com/ververica/flink-cdc-connectors seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/io/debezium/connector/sqlserver/SqlServerStreamingChangeEventSource.java from https://github.com/debezium/debezium +seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb from https://github.com/ververica/flink-cdc-connectors generate_client_protocol.sh from https://github.com/hazelcast/hazelcast seatunnel-engine/seatunnel-engine-common/src/main/java/org/apache/seatunnel/engine/common/utils/ExceptionUtil.java from https://github.com/hazelcast/hazelcast seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/protocol/task/AbstractSeaTunnelMessageTask.java from https://github.com/hazelcast/hazelcast @@ -239,4 +240,4 @@ seatunnel-api/src/main/java/org/apache/seatunnel/api/common/metrics seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sqlengine/zeta/ZetaSQLEngine.java from https://github.com/JSQLParser/JSqlParser seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sqlengine/zeta/ZetaSQLType.java from https://github.com/JSQLParser/JSqlParser seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sqlengine/zeta/ZetaSQLFilter.java from https://github.com/JSQLParser/JSqlParser -seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sqlengine/zeta/ZetaSQLFunction.java from https://github.com/JSQLParser/JSqlParser \ No newline at end of file +seatunnel-transforms-v2/src/main/java/org/apache/seatunnel/transform/sqlengine/zeta/ZetaSQLFunction.java from https://github.com/JSQLParser/JSqlParser diff --git a/config/plugin_config b/config/plugin_config index 98c9fa8c2c69..95b952b31bf1 100644 --- a/config/plugin_config +++ b/config/plugin_config @@ -24,6 +24,7 @@ connector-amazondynamodb connector-assert connector-cassandra connector-cdc-mysql +connector-cdc-mongodb connector-cdc-sqlserver connector-clickhouse connector-datahub diff --git a/docs/en/connector-v2/source/MongoDB-CDC.md b/docs/en/connector-v2/source/MongoDB-CDC.md new file mode 100644 index 000000000000..cb7c2f32acf5 --- /dev/null +++ b/docs/en/connector-v2/source/MongoDB-CDC.md @@ -0,0 +1,282 @@ +# MongoDB CDC + +> MongoDB CDC source connector + +## Support Those Engines + +> SeaTunnel Zeta
+ +## Key Features + +- [ ] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) + +## Description + +The MongoDB CDC connector allows for reading snapshot data and incremental data from MongoDB database. + +## Supported DataSource Info + +In order to use the Mongodb CDC connector, the following dependencies are required. +They can be downloaded via install-plugin.sh or from the Maven central repository. + +| Datasource | Supported Versions | Dependency | +|------------|--------------------|-------------------------------------------------------------------------------------------------------------------| +| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-cdc-mongodb) | + +## Availability Settings + +1.MongoDB version: MongoDB version >= 4.0. + +2.Cluster deployment: replica sets or sharded clusters. + +3.Storage Engine: WiredTiger Storage Engine. + +4.Permissions:changeStream and read + +```shell +use admin; +db.createRole( + { + role: "strole", + privileges: [{ + resource: { db: "", collection: "" }, + actions: [ + "splitVector", + "listDatabases", + "listCollections", + "collStats", + "find", + "changeStream" ] + }], + roles: [ + { role: 'read', db: 'config' } + ] + } +); + +db.createUser( + { + user: 'stuser', + pwd: 'stpw', + roles: [ + { role: 'strole', db: 'admin' } + ] + } +); +``` + +## Data Type Mapping + +The following table lists the field data type mapping from MongoDB BSON type to Seatunnel data type. + +| MongoDB BSON type | Seatunnel Data type | +|-------------------|---------------------| +| ObjectId | STRING | +| String | STRING | +| Boolean | BOOLEAN | +| Binary | BINARY | +| Int32 | INTEGER | +| Int64 | BIGINT | +| Double | DOUBLE | +| Decimal128 | DECIMAL | +| Date | Date | +| Timestamp | Timestamp | +| Object | ROW | +| Array | ARRAY | + +For specific types in MongoDB, we use Extended JSON format to map them to Seatunnel STRING type. + +| MongoDB BSON type | Seatunnel STRING | +|-------------------|----------------------------------------------------------------------------------------------| +| Symbol | {"_value": {"$symbol": "12"}} | +| RegularExpression | {"_value": {"$regularExpression": {"pattern": "^9$", "options": "i"}}} | +| JavaScript | {"_value": {"$code": "function() { return 10; }"}} | +| DbPointer | {"_value": {"$dbPointer": {"$ref": "db.coll", "$id": {"$oid": "63932a00da01604af329e33c"}}}} | + +**Tips** + +> 1.When using the DECIMAL type in SeaTunnel, be aware that the maximum range cannot exceed 34 digits, which means you should use decimal(34, 18).
+ +## Source Options + +| Name | Type | Required | Default | Description | +|------------------------------------|--------|----------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| hosts | String | Yes | - | The comma-separated list of hostname and port pairs of the MongoDB servers. eg. `localhost:27017,localhost:27018` | +| username | String | No | - | Name of the database user to be used when connecting to MongoDB. | +| password | String | No | - | Password to be used when connecting to MongoDB. | +| database | List | Yes | - | Name of the database to watch for changes. If not set then all databases will be captured. The database also supports regular expressions to monitor multiple databases matching the regular expression. eg. `db1,db2`. | +| collection | List | Yes | - | Name of the collection in the database to watch for changes. If not set then all collections will be captured. The collection also supports regular expressions to monitor multiple collections matching fully-qualified collection identifiers. eg. `db1.coll1,db2.coll2`. | +| connection.options | String | No | - | The ampersand-separated connection options of MongoDB. eg. `replicaSet=test&connectTimeoutMS=300000`. | +| batch.size | Long | No | 1024 | The cursor batch size. | +| poll.max.batch.size | Enum | No | 1024 | Maximum number of change stream documents to include in a single batch when polling for new data. | +| poll.await.time.ms | Long | No | 1000 | The amount of time to wait before checking for new results on the change stream. | +| heartbeat.interval.ms | String | No | 0 | The length of time in milliseconds between sending heartbeat messages. Use 0 to disable. | +| incremental.snapshot.chunk.size.mb | Long | No | 64 | The chunk size mb of incremental snapshot. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. | + +### Tips: + +> 1.If the collection changes at a slow pace, it is strongly recommended to set an appropriate value greater than 0 for the heartbeat.interval.ms parameter. When we recover a Seatunnel job from a checkpoint or savepoint, the heartbeat events can push the resumeToken forward to avoid its expiration.
+> 2.MongoDB has a limit of 16MB for a single document. Change documents include additional information, so even if the original document is not larger than 15MB, the change document may exceed the 16MB limit, resulting in the termination of the Change Stream operation.
+> 3.It is recommended to use immutable shard keys. In MongoDB, shard keys allow modifications after transactions are enabled, but changing the shard key can cause frequent shard migrations, resulting in additional performance overhead. Additionally, modifying the shard key can also cause the Update Lookup feature to become ineffective, leading to inconsistent results in CDC (Change Data Capture) scenarios.
+ +## How to Create a MongoDB CDC Data Synchronization Jobs + +### CDC Data Print to Client + +The following example demonstrates how to create a data synchronization job that reads cdc data from MongoDB and prints it on the local client: + +```hocon +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + execution.checkpoint.interval = 5000 +} + +source { + MongoDB-CDC { + hosts = "mongo0:27017" + database = ["inventory"] + collection = ["inventory.products"] + username = stuser + password = stpw + schema = { + fields { + "_id" : string, + "name" : string, + "description" : string, + "weight" : string + } + } + } +} + +# Console printing of the read Mongodb data +sink { + Console { + parallelism = 1 + } +} +``` + +## CDC Data Write to MysqlDB + +The following example demonstrates how to create a data synchronization job that reads cdc data from MongoDB and write to mysql database: + +```hocon +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + execution.checkpoint.interval = 5000 +} + +source { + MongoDB-CDC { + hosts = "mongo0:27017" + database = ["inventory"] + collection = ["inventory.products"] + username = stuser + password = stpw + } +} + +sink { + jdbc { + url = "jdbc:mysql://mysql_cdc_e2e:3306" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user" + password = "seatunnel" + + generate_sink_sql = true + # You need to configure both database and table + database = mongodb_cdc + table = products + primary_keys = ["_id"] + } +} +``` + +## Multi-table Synchronization + +The following example demonstrates how to create a data synchronization job that read the cdc data of multiple library tables mongodb and prints it on the local client: + +```hocon +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + execution.checkpoint.interval = 5000 +} + +source { + MongoDB-CDC { + hosts = "mongo0:27017" + database = ["inventory","crm"] + collection = ["inventory.products","crm.test"] + username = stuser + password = stpw + } +} + +# Console printing of the read Mongodb data +sink { + Console { + parallelism = 1 + } +} +``` + +### Tips: + +> 1.The cdc synchronization of multiple library tables cannot specify the schema, and can only output json data downstream. +> This is because MongoDB does not provide metadata information for querying, so if you want to support multiple tables, all tables can only be read as one structure. + +## Regular Expression Matching for Multiple Tables + +The following example demonstrates how to create a data synchronization job that through regular expression read the data of multiple library tables mongodb and prints it on the local client: + +| Matching example | Expressions | | Describe | +|------------------|-------------|---|----------------------------------------------------------------------------------------| +| Prefix matching | ^(test).* | | Match the database name or table name with the prefix test, such as test1, test2, etc. | +| Suffix matching | .*[p$] | | Match the database name or table name with the suffix p, such as cdcp, edcp, etc. | + +```hocon +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + execution.checkpoint.interval = 5000 +} + +source { + MongoDB-CDC { + hosts = "mongo0:27017" + # So this example is used (^(test).*|^(tpc).*|txc|.*[p$]|t{2}).(t[5-8]|tt),matching txc.tt、test2.test5. + database = ["(^(test).*|^(tpc).*|txc|.*[p$]|t{2})"] + collection = ["(t[5-8]|tt)"] + username = stuser + password = stpw + } +} + +# Console printing of the read Mongodb data +sink { + Console { + parallelism = 1 + } +} +``` + +## Changelog + +- [Feature]Add MongoDB CDC Source Connector([4923](https://github.com/apache/seatunnel/pull/4923)) + +### next version + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index de6593b4523c..3943159aefaf 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -99,6 +99,7 @@ seatunnel.sink.Doris = connector-doris seatunnel.source.Maxcompute = connector-maxcompute seatunnel.sink.Maxcompute = connector-maxcompute seatunnel.source.MySQL-CDC = connector-cdc-mysql +seatunnel.source.MongoDB-CDC = connector-cdc-mongodb seatunnel.sink.S3Redshift = connector-s3-redshift seatunnel.source.TDengine = connector-tdengine seatunnel.sink.TDengine = connector-tdengine diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java index 5257064dc1fe..c4d5fdfd68fe 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java @@ -28,7 +28,6 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.debezium.connector.base.ChangeEventQueue; import io.debezium.pipeline.DataChangeEvent; -import io.debezium.relational.TableId; import lombok.extern.slf4j.Slf4j; import java.util.ArrayList; @@ -39,8 +38,6 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import static org.apache.seatunnel.connectors.cdc.base.utils.SourceRecordUtils.getTableId; - /** * Fetcher to fetch data from table split, the split is the incremental split {@link * IncrementalSplit}. @@ -150,11 +147,14 @@ public void close() { private boolean shouldEmit(SourceRecord sourceRecord) { if (taskContext.isDataChangeRecord(sourceRecord)) { Offset position = taskContext.getStreamOffset(sourceRecord); - TableId tableId = getTableId(sourceRecord); + // TODO: The sourceRecord from MongoDB CDC and MySQL CDC are inconsistent. For + // compatibility, the getTableId method is commented out for now. + // TableId tableId = getTableId(sourceRecord); if (!taskContext.isExactlyOnce()) { - log.trace( - "The table {} is not support exactly-once, so ignore the watermark check", - tableId); + // log.trace( + // "The table {} is not support exactly-once, so ignore the + // watermark check", + // tableId); return position.isAfter(splitStartWatermark); } // TODO only the table who captured snapshot splits need to filter( Used to support diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/pom.xml b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/pom.xml new file mode 100644 index 000000000000..e22560ed0383 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/pom.xml @@ -0,0 +1,87 @@ + + + + 4.0.0 + + org.apache.seatunnel + connector-cdc + ${revision} + + connector-cdc-mongodb + SeaTunnel : Connectors V2 : CDC : Mongodb + + + 4.7.1 + 1.11.1 + 1.10.1 + 4.13.2 + + + + + org.apache.seatunnel + connector-cdc-base + ${project.version} + compile + + + io.debezium + debezium-connector-mongodb + ${debezium.version} + compile + + + org.mongodb.kafka + mongo-kafka-connect + ${mongo-kafka-connect.version} + + + org.mongodb + mongodb-driver-sync + + + org.apache.kafka + connect-api + + + org.apache.avro + avro + + + + + org.apache.avro + avro + ${avro.version} + + + org.mongodb + mongodb-driver-sync + ${mongo.driver.version} + + + junit + junit + ${junit.vserion} + test + + + diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSource.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSource.java new file mode 100644 index 000000000000..41191cfa52ba --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSource.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; +import org.apache.seatunnel.connectors.cdc.base.dialect.DataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.option.StopMode; +import org.apache.seatunnel.connectors.cdc.base.source.IncrementalSource; +import org.apache.seatunnel.connectors.cdc.base.source.offset.OffsetFactory; +import org.apache.seatunnel.connectors.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.seatunnel.connectors.cdc.debezium.row.DebeziumJsonDeserializeSchema; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfigProvider; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.sender.MongoDBConnectorDeserializationSchema; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.dialect.MongodbDialect; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffsetFactory; + +import com.google.auto.service.AutoService; +import lombok.NoArgsConstructor; + +import javax.annotation.Nonnull; + +import java.util.Optional; + +@NoArgsConstructor +@AutoService(SeaTunnelSource.class) +public class MongodbIncrementalSource extends IncrementalSource + implements SupportParallelism { + + static final String IDENTIFIER = "MongoDB-CDC"; + + public MongodbIncrementalSource( + ReadonlyConfig options, SeaTunnelDataType dataType) { + super(options, dataType); + } + + @Override + public Option getStartupModeOption() { + return MongodbSourceOptions.STARTUP_MODE; + } + + @Override + public Option getStopModeOption() { + return MongodbSourceOptions.STOP_MODE; + } + + @Override + public String getPluginName() { + return IDENTIFIER; + } + + @Override + public SourceConfig.Factory createSourceConfigFactory( + @Nonnull ReadonlyConfig config) { + MongodbSourceConfigProvider.Builder builder = + MongodbSourceConfigProvider.newBuilder() + .hosts(config.get(MongodbSourceOptions.HOSTS)) + .validate(); + Optional.ofNullable(config.get(MongodbSourceOptions.DATABASE)) + .ifPresent(builder::databaseList); + Optional.ofNullable(config.get(MongodbSourceOptions.COLLECTION)) + .ifPresent(builder::collectionList); + Optional.ofNullable(config.get(MongodbSourceOptions.USERNAME)).ifPresent(builder::username); + Optional.ofNullable(config.get(MongodbSourceOptions.PASSWORD)).ifPresent(builder::password); + Optional.ofNullable(config.get(MongodbSourceOptions.CONNECTION_OPTIONS)) + .ifPresent(builder::connectionOptions); + Optional.ofNullable(config.get(MongodbSourceOptions.BATCH_SIZE)) + .ifPresent(builder::batchSize); + Optional.ofNullable(config.get(MongodbSourceOptions.POLL_MAX_BATCH_SIZE)) + .ifPresent(builder::pollMaxBatchSize); + Optional.ofNullable(config.get(MongodbSourceOptions.POLL_AWAIT_TIME_MILLIS)) + .ifPresent(builder::pollAwaitTimeMillis); + Optional.ofNullable(config.get(MongodbSourceOptions.HEARTBEAT_INTERVAL_MILLIS)) + .ifPresent(builder::heartbeatIntervalMillis); + Optional.ofNullable(config.get(MongodbSourceOptions.HEARTBEAT_INTERVAL_MILLIS)) + .ifPresent(builder::splitMetaGroupSize); + Optional.ofNullable(config.get(MongodbSourceOptions.INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB)) + .ifPresent(builder::splitSizeMB); + Optional.ofNullable(startupConfig).ifPresent(builder::startupOptions); + Optional.ofNullable(stopConfig).ifPresent(builder::stopOptions); + return builder; + } + + @SuppressWarnings("unchecked") + @Override + public DebeziumDeserializationSchema createDebeziumDeserializationSchema( + ReadonlyConfig config) { + SeaTunnelDataType physicalRowType; + if (dataType == null) { + return (DebeziumDeserializationSchema) + new DebeziumJsonDeserializeSchema( + config.get(MongodbSourceOptions.DEBEZIUM_PROPERTIES)); + } else { + physicalRowType = dataType; + return (DebeziumDeserializationSchema) + new MongoDBConnectorDeserializationSchema(physicalRowType, physicalRowType); + } + } + + @Override + public DataSourceDialect createDataSourceDialect(ReadonlyConfig config) { + return new MongodbDialect(); + } + + @Override + public OffsetFactory createOffsetFactory(ReadonlyConfig config) { + return new ChangeStreamOffsetFactory(); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSourceFactory.java new file mode 100644 index 000000000000..6215afb74ef0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSourceFactory.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.SupportMultipleTable; +import org.apache.seatunnel.api.table.factory.TableFactoryContext; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.type.MultipleRowType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions; + +import com.google.auto.service.AutoService; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +@AutoService(Factory.class) +public class MongodbIncrementalSourceFactory implements TableSourceFactory, SupportMultipleTable { + @Override + public String factoryIdentifier() { + return MongodbIncrementalSource.IDENTIFIER; + } + + @Override + public OptionRule optionRule() { + return MongodbSourceOptions.getBaseRule() + .required( + MongodbSourceOptions.HOSTS, + MongodbSourceOptions.DATABASE, + MongodbSourceOptions.COLLECTION) + .optional( + MongodbSourceOptions.USERNAME, + MongodbSourceOptions.PASSWORD, + MongodbSourceOptions.CONNECTION_OPTIONS, + MongodbSourceOptions.BATCH_SIZE, + MongodbSourceOptions.POLL_MAX_BATCH_SIZE, + MongodbSourceOptions.POLL_AWAIT_TIME_MILLIS, + MongodbSourceOptions.HEARTBEAT_INTERVAL_MILLIS, + MongodbSourceOptions.INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB, + MongodbSourceOptions.STARTUP_MODE, + MongodbSourceOptions.STOP_MODE) + .build(); + } + + @Override + public Class getSourceClass() { + return MongodbIncrementalSource.class; + } + + @SuppressWarnings("unchecked") + @Override + public + TableSource createSource(TableFactoryContext context) { + return () -> { + SeaTunnelDataType dataType; + if (context.getCatalogTables().size() == 1) { + dataType = + context.getCatalogTables().get(0).getTableSchema().toPhysicalRowDataType(); + } else { + Map rowTypeMap = new HashMap<>(); + for (CatalogTable catalogTable : context.getCatalogTables()) { + rowTypeMap.put( + catalogTable.getTableId().toTablePath().toString(), + catalogTable.getTableSchema().toPhysicalRowDataType()); + } + dataType = new MultipleRowType(rowTypeMap); + } + return (SeaTunnelSource) + new MongodbIncrementalSource<>(context.getOptions(), dataType); + }; + } + + @Override + public Result applyTables(@Nonnull TableFactoryContext context) { + return Result.of(context.getCatalogTables(), Collections.emptyList()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfig.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfig.java new file mode 100644 index 000000000000..049b37db3634 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfig.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config; + +import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StartupConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StopConfig; + +import lombok.EqualsAndHashCode; +import lombok.Getter; + +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.buildConnectionString; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; + +@Getter +@EqualsAndHashCode +public class MongodbSourceConfig implements SourceConfig { + + private static final long serialVersionUID = 1L; + + private final String hosts; + + private final String username; + + private final String password; + + private final List databaseList; + + private final List collectionList; + + private final String connectionString; + + private final int batchSize; + + private final int pollAwaitTimeMillis; + + private final int pollMaxBatchSize; + + private final boolean updateLookup; + + private final StartupConfig startupOptions; + + private final StopConfig stopOptions; + + private final int heartbeatIntervalMillis; + + private final int splitMetaGroupSize; + + private final int splitSizeMB; + + MongodbSourceConfig( + String hosts, + String username, + String password, + List databaseList, + List collectionList, + String connectionOptions, + int batchSize, + int pollAwaitTimeMillis, + int pollMaxBatchSize, + boolean updateLookup, + StartupConfig startupOptions, + StopConfig stopOptions, + int heartbeatIntervalMillis, + int splitMetaGroupSize, + int splitSizeMB) { + this.hosts = checkNotNull(hosts); + this.username = username; + this.password = password; + this.databaseList = databaseList; + this.collectionList = collectionList; + this.connectionString = + buildConnectionString(username, password, hosts, connectionOptions) + .getConnectionString(); + this.batchSize = batchSize; + this.pollAwaitTimeMillis = pollAwaitTimeMillis; + this.pollMaxBatchSize = pollMaxBatchSize; + this.updateLookup = updateLookup; + this.startupOptions = startupOptions; + this.stopOptions = stopOptions; + this.heartbeatIntervalMillis = heartbeatIntervalMillis; + this.splitMetaGroupSize = splitMetaGroupSize; + this.splitSizeMB = splitSizeMB; + } + + @Override + public StartupConfig getStartupConfig() { + return startupOptions; + } + + @Override + public StopConfig getStopConfig() { + return stopOptions; + } + + @Override + public int getSplitSize() { + return splitSizeMB; + } + + @Override + public boolean isExactlyOnce() { + return true; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfigProvider.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfigProvider.java new file mode 100644 index 000000000000..ebe7af13e0c6 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfigProvider.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config; + +import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StartupConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StopConfig; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.option.StopMode; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import java.util.List; +import java.util.Objects; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.BATCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.HEARTBEAT_INTERVAL_MILLIS; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.POLL_AWAIT_TIME_MILLIS; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.POLL_MAX_BATCH_SIZE; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkArgument; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; + +public class MongodbSourceConfigProvider { + + private MongodbSourceConfigProvider() {} + + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder implements SourceConfig.Factory { + private String hosts; + private String username; + private String password; + private List databaseList; + private List collectionList; + private String connectionOptions; + private int batchSize = BATCH_SIZE.defaultValue(); + private int pollAwaitTimeMillis = POLL_AWAIT_TIME_MILLIS.defaultValue(); + private int pollMaxBatchSize = POLL_MAX_BATCH_SIZE.defaultValue(); + private StartupConfig startupOptions; + private StopConfig stopOptions; + private int heartbeatIntervalMillis = HEARTBEAT_INTERVAL_MILLIS.defaultValue(); + private int splitMetaGroupSize = 2; + private int splitSizeMB = INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB.defaultValue(); + + public Builder hosts(String hosts) { + this.hosts = hosts; + return this; + } + + public Builder connectionOptions(String connectionOptions) { + this.connectionOptions = connectionOptions; + return this; + } + + public Builder username(String username) { + this.username = username; + return this; + } + + public Builder password(String password) { + this.password = password; + return this; + } + + public Builder databaseList(List databases) { + this.databaseList = databases; + return this; + } + + public Builder collectionList(List collections) { + this.collectionList = collections; + return this; + } + + public Builder batchSize(int batchSize) { + checkArgument(batchSize >= 0); + this.batchSize = batchSize; + return this; + } + + public Builder pollAwaitTimeMillis(int pollAwaitTimeMillis) { + checkArgument(pollAwaitTimeMillis > 0); + this.pollAwaitTimeMillis = pollAwaitTimeMillis; + return this; + } + + public Builder pollMaxBatchSize(int pollMaxBatchSize) { + checkArgument(pollMaxBatchSize > 0); + this.pollMaxBatchSize = pollMaxBatchSize; + return this; + } + + public Builder startupOptions(StartupConfig startupOptions) { + this.startupOptions = Objects.requireNonNull(startupOptions); + if (startupOptions.getStartupMode() != StartupMode.INITIAL + && startupOptions.getStartupMode() != StartupMode.TIMESTAMP) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unsupported startup mode " + startupOptions.getStartupMode()); + } + return this; + } + + public Builder stopOptions(StopConfig stopOptions) { + this.stopOptions = Objects.requireNonNull(stopOptions); + if (stopOptions.getStopMode() != StopMode.NEVER) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + String.format("The %s mode is not supported.", stopOptions.getStopMode())); + } + return this; + } + + public Builder heartbeatIntervalMillis(int heartbeatIntervalMillis) { + checkArgument(heartbeatIntervalMillis >= 0); + this.heartbeatIntervalMillis = heartbeatIntervalMillis; + return this; + } + + public Builder splitSizeMB(int splitSizeMB) { + checkArgument(splitSizeMB > 0); + this.splitSizeMB = splitSizeMB; + return this; + } + + public Builder splitMetaGroupSize(int splitMetaGroupSize) { + this.splitMetaGroupSize = splitMetaGroupSize; + return this; + } + + public Builder validate() { + checkNotNull(hosts, "hosts must be provided"); + return this; + } + + @Override + public MongodbSourceConfig create(int subtask) { + boolean updateLookup = true; + return new MongodbSourceConfig( + hosts, + username, + password, + databaseList, + collectionList, + connectionOptions, + batchSize, + pollAwaitTimeMillis, + pollMaxBatchSize, + updateLookup, + startupOptions, + stopOptions, + heartbeatIntervalMillis, + splitMetaGroupSize, + splitSizeMB); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java new file mode 100644 index 000000000000..df73772e0718 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.connectors.cdc.base.option.SourceOptions; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.option.StopMode; + +import org.bson.BsonDouble; +import org.bson.json.JsonMode; +import org.bson.json.JsonWriterSettings; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MongodbSourceOptions extends SourceOptions { + + public static final String ENCODE_VALUE_FIELD = "_value"; + + public static final String CLUSTER_TIME_FIELD = "clusterTime"; + + public static final String TS_MS_FIELD = "ts_ms"; + + public static final String SOURCE_FIELD = "source"; + + public static final String SNAPSHOT_FIELD = "snapshot"; + + public static final String FALSE_FALSE = "false"; + + public static final String OPERATION_TYPE_INSERT = "insert"; + + public static final String SNAPSHOT_TRUE = "true"; + + public static final String ID_FIELD = "_id"; + + public static final String DOCUMENT_KEY = "documentKey"; + + public static final String NS_FIELD = "ns"; + + public static final String OPERATION_TYPE = "operationType"; + + public static final String TIMESTAMP_FIELD = "timestamp"; + + public static final String RESUME_TOKEN_FIELD = "resumeToken"; + + public static final String FULL_DOCUMENT = "fullDocument"; + + public static final String DB_FIELD = "db"; + + public static final String COLL_FIELD = "coll"; + + public static final int FAILED_TO_PARSE_ERROR = 9; + + public static final int UNAUTHORIZED_ERROR = 13; + + public static final int ILLEGAL_OPERATION_ERROR = 20; + + public static final int UNKNOWN_FIELD_ERROR = 40415; + + public static final String DROPPED_FIELD = "dropped"; + + public static final String MAX_FIELD = "max"; + + public static final String MIN_FIELD = "min"; + + public static final String ADD_NS_FIELD_NAME = "_ns_"; + + public static final String UUID_FIELD = "uuid"; + + public static final String SHARD_FIELD = "shard"; + + public static final String DIALECT_NAME = "MongoDB"; + + public static final BsonDouble COMMAND_SUCCEED_FLAG = new BsonDouble(1.0d); + + public static final JsonWriterSettings DEFAULT_JSON_WRITER_SETTINGS = + JsonWriterSettings.builder().outputMode(JsonMode.EXTENDED).build(); + + public static final String OUTPUT_SCHEMA = + "{" + + " \"name\": \"ChangeStream\"," + + " \"type\": \"record\"," + + " \"fields\": [" + + " { \"name\": \"_id\", \"type\": \"string\" }," + + " { \"name\": \"operationType\", \"type\": [\"string\", \"null\"] }," + + " { \"name\": \"fullDocument\", \"type\": [\"string\", \"null\"] }," + + " { \"name\": \"source\"," + + " \"type\": [{\"name\": \"source\", \"type\": \"record\", \"fields\": [" + + " {\"name\": \"ts_ms\", \"type\": \"long\"}," + + " {\"name\": \"snapshot\", \"type\": [\"string\", \"null\"] } ]" + + " }, \"null\" ] }," + + " { \"name\": \"ts_ms\", \"type\": [\"long\", \"null\"]}," + + " { \"name\": \"ns\"," + + " \"type\": [{\"name\": \"ns\", \"type\": \"record\", \"fields\": [" + + " {\"name\": \"db\", \"type\": \"string\"}," + + " {\"name\": \"coll\", \"type\": [\"string\", \"null\"] } ]" + + " }, \"null\" ] }," + + " { \"name\": \"to\"," + + " \"type\": [{\"name\": \"to\", \"type\": \"record\", \"fields\": [" + + " {\"name\": \"db\", \"type\": \"string\"}," + + " {\"name\": \"coll\", \"type\": [\"string\", \"null\"] } ]" + + " }, \"null\" ] }," + + " { \"name\": \"documentKey\", \"type\": [\"string\", \"null\"] }," + + " { \"name\": \"updateDescription\"," + + " \"type\": [{\"name\": \"updateDescription\", \"type\": \"record\", \"fields\": [" + + " {\"name\": \"updatedFields\", \"type\": [\"string\", \"null\"]}," + + " {\"name\": \"removedFields\"," + + " \"type\": [{\"type\": \"array\", \"items\": \"string\"}, \"null\"]" + + " }] }, \"null\"] }," + + " { \"name\": \"clusterTime\", \"type\": [\"string\", \"null\"] }," + + " { \"name\": \"txnNumber\", \"type\": [\"long\", \"null\"]}," + + " { \"name\": \"lsid\", \"type\": [{\"name\": \"lsid\", \"type\": \"record\"," + + " \"fields\": [ {\"name\": \"id\", \"type\": \"string\"}," + + " {\"name\": \"uid\", \"type\": \"string\"}] }, \"null\"] }" + + " ]" + + "}"; + + public static final Option HOSTS = + Options.key("hosts") + .stringType() + .noDefaultValue() + .withDescription( + "The comma-separated list of hostname and port pairs of the MongoDB servers. " + + "eg. localhost:27017,localhost:27018"); + + public static final Option USERNAME = + Options.key("username") + .stringType() + .noDefaultValue() + .withDescription( + "Name of the database user to be used when connecting to MongoDB. " + + "This is required only when MongoDB is configured to use authentication."); + + public static final Option PASSWORD = + Options.key("password") + .stringType() + .noDefaultValue() + .withDescription( + "Password to be used when connecting to MongoDB. " + + "This is required only when MongoDB is configured to use authentication."); + + public static final Option> DATABASE = + Options.key("database") + .listType() + .noDefaultValue() + .withDescription("Name of the database to watch for changes."); + + public static final Option> COLLECTION = + Options.key("collection") + .listType() + .noDefaultValue() + .withDescription( + "Name of the collection in the database to watch for changes."); + + public static final Option CONNECTION_OPTIONS = + Options.key("connection.options") + .stringType() + .noDefaultValue() + .withDescription( + "The ampersand-separated MongoDB connection options. " + + "eg. replicaSet=test&connectTimeoutMS=300000"); + + public static final Option BATCH_SIZE = + Options.key("batch.size") + .intType() + .defaultValue(1024) + .withDescription("The cursor batch size. Defaults to 1024."); + + public static final Option POLL_MAX_BATCH_SIZE = + Options.key("poll.max.batch.size") + .intType() + .defaultValue(1024) + .withDescription( + "Maximum number of change stream documents " + + "to include in a single batch when polling for new data. " + + "This setting can be used to limit the amount of data buffered internally in the connector. " + + "Defaults to 1024."); + + public static final Option POLL_AWAIT_TIME_MILLIS = + Options.key("poll.await.time.ms") + .intType() + .defaultValue(1000) + .withDescription( + "The amount of time to wait before checking for new results on the change stream." + + "Defaults: 1000."); + + public static final Option HEARTBEAT_INTERVAL_MILLIS = + Options.key("heartbeat.interval.ms") + .intType() + .defaultValue(0) + .withDescription( + "The length of time in milliseconds between sending heartbeat messages." + + "Heartbeat messages contain the post batch resume token and are sent when no source records " + + "have been published in the specified interval. This improves the resumability of the connector " + + "for low volume namespaces. Use 0 to disable. Defaults to 0."); + + public static final Option INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB = + Options.key("incremental.snapshot.chunk.size.mb") + .intType() + .defaultValue(64) + .withDescription( + "The chunk size mb of incremental snapshot. Defaults to 64mb."); + + public static final Option> DEBEZIUM_PROPERTIES = + Options.key("debezium") + .mapType() + .defaultValue( + new HashMap() { + { + put("key.converter.schemas.enable", "false"); + put("value.converter.schemas.enable", "false"); + } + }) + .withDescription( + "Decides if the table options contains Debezium client properties that start with prefix 'debezium'."); + + public static final Option STARTUP_MODE = + Options.key(SourceOptions.STARTUP_MODE_KEY) + .singleChoice( + StartupMode.class, + Arrays.asList( + StartupMode.INITIAL, StartupMode.EARLIEST, StartupMode.LATEST)) + .defaultValue(StartupMode.INITIAL) + .withDescription( + "Optional startup mode for CDC source, valid enumerations are " + + "\"initial\", \"earliest\", \"latest\", \"timestamp\"\n or \"specific\""); + + public static final Option STOP_MODE = + Options.key(SourceOptions.STOP_MODE_KEY) + .singleChoice(StopMode.class, Collections.singletonList(StopMode.NEVER)) + .defaultValue(StopMode.NEVER) + .withDescription( + "Optional stop mode for CDC source, valid enumerations are " + + "\"never\", \"latest\", \"timestamp\"\n or \"specific\""); +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/exception/MongodbConnectorException.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/exception/MongodbConnectorException.java new file mode 100644 index 000000000000..2d2267e478cc --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/exception/MongodbConnectorException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +public class MongodbConnectorException extends SeaTunnelRuntimeException { + + public MongodbConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { + super(seaTunnelErrorCode, errorMessage); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/internal/MongodbClientProvider.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/internal/MongodbClientProvider.java new file mode 100644 index 000000000000..dc621ed9dc00 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/internal/MongodbClientProvider.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.internal; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; + +import com.mongodb.ConnectionString; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public enum MongodbClientProvider { + INSTANCE; + + private volatile MongoClient mongoClient; + + public MongoClient getOrCreateMongoClient(MongodbSourceConfig sourceConfig) { + if (mongoClient == null) { + ConnectionString connectionString = + new ConnectionString(sourceConfig.getConnectionString()); + log.info( + "Create and register mongo client {}@{}", + connectionString.getUsername(), + connectionString.getHosts()); + mongoClient = MongoClients.create(connectionString); + } + return mongoClient; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/MongoDBConnectorDeserializationSchema.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/MongoDBConnectorDeserializationSchema.java new file mode 100644 index 000000000000..75f3564c6c6a --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/MongoDBConnectorDeserializationSchema.java @@ -0,0 +1,553 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.sender; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.connectors.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonType; +import org.bson.BsonValue; +import org.bson.json.JsonMode; +import org.bson.json.JsonWriterSettings; +import org.bson.types.Decimal128; + +import com.mongodb.client.model.changestream.OperationType; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_DATA_TYPE; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_OPERATION; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DEFAULT_JSON_WRITER_SETTINGS; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ENCODE_VALUE_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.FULL_DOCUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; + +public class MongoDBConnectorDeserializationSchema + implements DebeziumDeserializationSchema { + + private final SeaTunnelDataType resultTypeInfo; + + private final DeserializationRuntimeConverter physicalConverter; + + public MongoDBConnectorDeserializationSchema( + SeaTunnelDataType physicalDataType, + SeaTunnelDataType resultTypeInfo) { + this.physicalConverter = createConverter(physicalDataType); + this.resultTypeInfo = resultTypeInfo; + } + + @Override + public void deserialize(@Nonnull SourceRecord record, Collector out) { + Struct value = (Struct) record.value(); + Schema valueSchema = record.valueSchema(); + + OperationType op = operationTypeFor(record); + BsonDocument documentKey = + checkNotNull( + Objects.requireNonNull( + extractBsonDocument(value, valueSchema, DOCUMENT_KEY))); + BsonDocument fullDocument = extractBsonDocument(value, valueSchema, FULL_DOCUMENT); + + switch (op) { + case INSERT: + SeaTunnelRow insert = extractRowData(fullDocument); + insert.setRowKind(RowKind.INSERT); + emit(record, insert, out); + break; + case DELETE: + SeaTunnelRow delete = + new SeaTunnelRow( + new Object[] { + documentKey.get(ID_FIELD).asObjectId().getValue().toString() + }); + delete.setRowKind(RowKind.DELETE); + emit(record, delete, out); + break; + case UPDATE: + if (fullDocument == null) { + break; + } + SeaTunnelRow updateAfter = extractRowData(fullDocument); + updateAfter.setRowKind(RowKind.UPDATE_AFTER); + emit(record, updateAfter, out); + break; + case REPLACE: + SeaTunnelRow replaceAfter = extractRowData(fullDocument); + replaceAfter.setRowKind(RowKind.UPDATE_AFTER); + emit(record, replaceAfter, out); + break; + case INVALIDATE: + case DROP: + case DROP_DATABASE: + case RENAME: + case OTHER: + default: + break; + } + } + + @Override + public SeaTunnelDataType getProducedType() { + return resultTypeInfo; + } + + private @Nonnull OperationType operationTypeFor(@Nonnull SourceRecord record) { + Struct value = (Struct) record.value(); + return OperationType.fromString(value.getString("operationType")); + } + + // TODO:The dynamic schema will be completed based on this method later. + private void emit( + SourceRecord inRecord, + SeaTunnelRow physicalRow, + @Nonnull Collector collector) { + collector.collect(physicalRow); + } + + private SeaTunnelRow extractRowData(BsonDocument document) { + checkNotNull(document); + return (SeaTunnelRow) physicalConverter.convert(document); + } + + private BsonDocument extractBsonDocument( + Struct value, @Nonnull Schema valueSchema, String fieldName) { + if (valueSchema.field(fieldName) != null) { + String docString = value.getString(fieldName); + if (docString != null) { + return BsonDocument.parse(docString); + } + } + return null; + } + + // ------------------------------------------------------------------------------------- + // Runtime Converters + // ------------------------------------------------------------------------------------- + + @FunctionalInterface + public interface DeserializationRuntimeConverter extends Serializable { + Object convert(BsonValue bsonValue); + } + + public DeserializationRuntimeConverter createConverter(SeaTunnelDataType type) { + SerializableFunction internalRowConverter = + createNullSafeInternalConverter(type); + return new DeserializationRuntimeConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(BsonValue bsonValue) { + return internalRowConverter.apply(bsonValue); + } + }; + } + + private static SerializableFunction createNullSafeInternalConverter( + SeaTunnelDataType type) { + return wrapIntoNullSafeInternalConverter(createInternalConverter(type), type); + } + + private static SerializableFunction wrapIntoNullSafeInternalConverter( + SerializableFunction internalConverter, SeaTunnelDataType type) { + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + if (isBsonValueNull(bsonValue) || isBsonDecimalNaN(bsonValue)) { + throw new MongodbConnectorException( + UNSUPPORTED_OPERATION, + "Unable to convert to <" + type + "> from nullable value " + bsonValue); + } + return internalConverter.apply(bsonValue); + } + }; + } + + private static boolean isBsonValueNull(BsonValue bsonValue) { + return bsonValue == null + || bsonValue.isNull() + || bsonValue.getBsonType() == BsonType.UNDEFINED; + } + + private static boolean isBsonDecimalNaN(@Nonnull BsonValue bsonValue) { + return bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN(); + } + + private static SerializableFunction createInternalConverter( + @Nonnull SeaTunnelDataType type) { + switch (type.getSqlType()) { + case NULL: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + return null; + } + }; + case BOOLEAN: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + return convertToBoolean(bsonValue); + } + }; + case DOUBLE: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + return convertToDouble(bsonValue); + } + }; + case INT: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + return convertToInt(bsonValue); + } + }; + case BIGINT: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + return convertToLong(bsonValue); + } + }; + case BYTES: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + return convertToBinary(bsonValue); + } + }; + case STRING: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + return convertToString(bsonValue); + } + }; + case DATE: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + return convertToLocalDateTime(bsonValue).toLocalDate(); + } + }; + case TIMESTAMP: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + return convertToLocalDateTime(bsonValue); + } + }; + case DECIMAL: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + DecimalType decimalType = (DecimalType) type; + BigDecimal decimalValue = convertToBigDecimal(bsonValue); + return fromBigDecimal( + decimalValue, decimalType.getPrecision(), decimalType.getScale()); + } + }; + case ARRAY: + return createArrayConverter((ArrayType) type); + case MAP: + MapType mapType = (MapType) type; + return createMapConverter( + mapType.toString(), mapType.getKeyType(), mapType.getValueType()); + + case ROW: + return createRowConverter((SeaTunnelRowType) type); + default: + throw new MongodbConnectorException( + UNSUPPORTED_DATA_TYPE, "Not support to parse type: " + type); + } + } + + private static LocalDateTime convertToLocalDateTime(BsonValue bsonValue) { + Instant instant; + if (bsonValue.isTimestamp()) { + instant = Instant.ofEpochSecond(bsonValue.asTimestamp().getTime()); + } else if (bsonValue.isDateTime()) { + instant = Instant.ofEpochMilli(bsonValue.asDateTime().getValue()); + } else { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert to LocalDateTime from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + return Timestamp.from(instant).toLocalDateTime(); + } + + @SuppressWarnings("unchecked") + private static SerializableFunction createRowConverter( + SeaTunnelRowType type) { + SeaTunnelDataType[] fieldTypes = type.getFieldTypes(); + final SerializableFunction[] fieldConverters = + Arrays.stream(fieldTypes) + .map(MongoDBConnectorDeserializationSchema::createNullSafeInternalConverter) + .toArray(SerializableFunction[]::new); + int fieldCount = type.getTotalFields(); + + final String[] fieldNames = type.getFieldNames(); + + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + if (!bsonValue.isDocument()) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert to rowType from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + + BsonDocument document = bsonValue.asDocument(); + SeaTunnelRow row = new SeaTunnelRow(fieldCount); + for (int i = 0; i < fieldCount; i++) { + String fieldName = fieldNames[i]; + BsonValue fieldValue = document.get(fieldName); + Object convertedField = fieldConverters[i].apply(fieldValue); + row.setField(i, convertedField); + } + return row; + } + }; + } + + private static @Nonnull SerializableFunction createArrayConverter( + @Nonnull ArrayType type) { + final SerializableFunction elementConverter = + createNullSafeInternalConverter(type.getElementType()); + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + if (!bsonValue.isArray()) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert to arrayType from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + + List in = bsonValue.asArray(); + Object arr = Array.newInstance(type.getElementType().getTypeClass(), in.size()); + for (int i = 0; i < in.size(); i++) { + Array.set(arr, i, elementConverter.apply(in.get(i))); + } + return arr; + } + }; + } + + private static @Nonnull SerializableFunction createMapConverter( + String typeSummary, + @Nonnull SeaTunnelDataType keyType, + SeaTunnelDataType valueType) { + if (!keyType.getSqlType().equals(SqlType.STRING)) { + throw new MongodbConnectorException( + UNSUPPORTED_OPERATION, + "Bson format doesn't support non-string as key type of map. The type is: " + + typeSummary); + } + SerializableFunction valueConverter = + createNullSafeInternalConverter(valueType); + + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + if (!bsonValue.isDocument()) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert to rowType from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + + BsonDocument document = bsonValue.asDocument(); + Map map = new HashMap<>(); + for (String key : document.keySet()) { + map.put(key, valueConverter.apply(document.get(key))); + } + return map; + } + }; + } + + public static BigDecimal fromBigDecimal(BigDecimal bd, int precision, int scale) { + bd = bd.setScale(scale, RoundingMode.HALF_UP); + if (bd.precision() > precision) { + return null; + } + return bd; + } + + private static boolean convertToBoolean(@Nonnull BsonValue bsonValue) { + if (bsonValue.isBoolean()) { + return bsonValue.asBoolean().getValue(); + } + throw new MongodbConnectorException( + UNSUPPORTED_DATA_TYPE, + "Unable to convert to boolean from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + + private static double convertToDouble(@Nonnull BsonValue bsonValue) { + if (bsonValue.isDouble()) { + return bsonValue.asNumber().doubleValue(); + } + throw new MongodbConnectorException( + UNSUPPORTED_DATA_TYPE, + "Unable to convert to double from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + + private static int convertToInt(@Nonnull BsonValue bsonValue) { + if (bsonValue.isInt32()) { + return bsonValue.asNumber().intValue(); + } + throw new MongodbConnectorException( + UNSUPPORTED_DATA_TYPE, + "Unable to convert to integer from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + + private static String convertToString(@Nonnull BsonValue bsonValue) { + if (bsonValue.isString()) { + return bsonValue.asString().getValue(); + } + if (bsonValue.isObjectId()) { + return bsonValue.asObjectId().getValue().toHexString(); + } + if (bsonValue.isDocument()) { + return bsonValue + .asDocument() + .toJson(JsonWriterSettings.builder().outputMode(JsonMode.RELAXED).build()); + } + return new BsonDocument(ENCODE_VALUE_FIELD, bsonValue).toJson(DEFAULT_JSON_WRITER_SETTINGS); + } + + private static byte[] convertToBinary(@Nonnull BsonValue bsonValue) { + if (bsonValue.isBinary()) { + return bsonValue.asBinary().getData(); + } + throw new MongodbConnectorException( + UNSUPPORTED_DATA_TYPE, + "Unsupported BYTES value type: " + bsonValue.getClass().getSimpleName()); + } + + private static long convertToLong(@Nonnull BsonValue bsonValue) { + if (bsonValue.isInt64()) { + return bsonValue.asNumber().longValue(); + } + throw new MongodbConnectorException( + UNSUPPORTED_DATA_TYPE, + "Unable to convert to long from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + + private static BigDecimal convertToBigDecimal(@Nonnull BsonValue bsonValue) { + if (bsonValue.isDecimal128()) { + Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value(); + if (decimal128Value.isFinite()) { + return bsonValue.asDecimal128().decimal128Value().bigDecimalValue(); + } else { + // DecimalData doesn't have the concept of infinity. + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert infinite bson decimal to Decimal type."); + } + } + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert to decimal from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/SerializableFunction.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/SerializableFunction.java new file mode 100644 index 000000000000..8905fbb02739 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/SerializableFunction.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.sender; + +import java.io.Serializable; +import java.util.function.Function; + +@FunctionalInterface +public interface SerializableFunction extends Function, Serializable {} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java new file mode 100644 index 000000000000..11ef57ffc5f9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.dialect; + +import org.apache.seatunnel.connectors.cdc.base.dialect.DataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch.MongodbFetchTaskContext; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch.MongodbScanFetchTask; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch.MongodbStreamFetchTask; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamDescriptor; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters.MongodbChunkSplitter; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils; + +import org.bson.BsonDocument; + +import com.mongodb.client.MongoClient; +import io.debezium.relational.TableId; + +import javax.annotation.Nonnull; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DIALECT_NAME; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.collectionNames; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.collectionsFilter; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.databaseFilter; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.databaseNames; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.createMongoClient; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getChangeStreamDescriptor; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getCurrentClusterTime; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getLatestResumeToken; + +public class MongodbDialect implements DataSourceDialect { + + private final Map cache = + new ConcurrentHashMap<>(); + + @Override + public String getName() { + return DIALECT_NAME; + } + + @Override + public List discoverDataCollections(MongodbSourceConfig sourceConfig) { + CollectionDiscoveryUtils.CollectionDiscoveryInfo discoveryInfo = + discoverAndCacheDataCollections(sourceConfig); + return discoveryInfo.getDiscoveredCollections().stream() + .map(TableId::parse) + .collect(Collectors.toList()); + } + + @Override + public boolean isDataCollectionIdCaseSensitive(MongodbSourceConfig sourceConfig) { + // MongoDB's database names and collection names are case-sensitive. + return true; + } + + @Override + public ChunkSplitter createChunkSplitter(MongodbSourceConfig sourceConfig) { + return new MongodbChunkSplitter(sourceConfig); + } + + @Override + public FetchTask createFetchTask(@Nonnull SourceSplitBase sourceSplitBase) { + if (sourceSplitBase.isSnapshotSplit()) { + return new MongodbScanFetchTask(sourceSplitBase.asSnapshotSplit()); + } else { + return new MongodbStreamFetchTask(sourceSplitBase.asIncrementalSplit()); + } + } + + @Override + public FetchTask.Context createFetchTaskContext( + SourceSplitBase sourceSplitBase, MongodbSourceConfig sourceConfig) { + CollectionDiscoveryUtils.CollectionDiscoveryInfo discoveryInfo = + discoverAndCacheDataCollections(sourceConfig); + ChangeStreamDescriptor changeStreamDescriptor = + getChangeStreamDescriptor( + sourceConfig, + discoveryInfo.getDiscoveredDatabases(), + discoveryInfo.getDiscoveredCollections()); + return new MongodbFetchTaskContext(this, sourceConfig, changeStreamDescriptor); + } + + private CollectionDiscoveryUtils.CollectionDiscoveryInfo discoverAndCacheDataCollections( + MongodbSourceConfig sourceConfig) { + return cache.computeIfAbsent( + sourceConfig, + config -> { + MongoClient mongoClient = createMongoClient(sourceConfig); + List discoveredDatabases = + databaseNames( + mongoClient, databaseFilter(sourceConfig.getDatabaseList())); + List discoveredCollections = + collectionNames( + mongoClient, + discoveredDatabases, + collectionsFilter(sourceConfig.getCollectionList())); + return new CollectionDiscoveryUtils.CollectionDiscoveryInfo( + discoveredDatabases, discoveredCollections); + }); + } + + public ChangeStreamOffset displayCurrentOffset(MongodbSourceConfig sourceConfig) { + MongoClient mongoClient = createMongoClient(sourceConfig); + CollectionDiscoveryUtils.CollectionDiscoveryInfo discoveryInfo = + discoverAndCacheDataCollections(sourceConfig); + ChangeStreamDescriptor changeStreamDescriptor = + getChangeStreamDescriptor( + sourceConfig, + discoveryInfo.getDiscoveredDatabases(), + discoveryInfo.getDiscoveredCollections()); + BsonDocument startupResumeToken = getLatestResumeToken(mongoClient, changeStreamDescriptor); + + ChangeStreamOffset changeStreamOffset; + if (startupResumeToken != null) { + changeStreamOffset = new ChangeStreamOffset(startupResumeToken); + } else { + changeStreamOffset = new ChangeStreamOffset(getCurrentClusterTime(mongoClient)); + } + + return changeStreamOffset; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbFetchTaskContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbFetchTaskContext.java new file mode 100644 index 000000000000..534baa72abda --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbFetchTaskContext.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch; + +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.dialect.MongodbDialect; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamDescriptor; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils; + +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonType; +import org.bson.BsonValue; + +import com.mongodb.client.model.changestream.OperationType; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.pipeline.DataChangeEvent; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.util.LoggingContext; + +import javax.annotation.Nonnull; + +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.OPERATION_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_TRUE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SOURCE_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.TS_MS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.BsonUtils.compareBsonValue; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.getDocumentKey; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.getResumeToken; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.createMongoClient; + +public class MongodbFetchTaskContext implements FetchTask.Context { + + private final MongodbDialect dialect; + private final MongodbSourceConfig sourceConfig; + private final ChangeStreamDescriptor changeStreamDescriptor; + private ChangeEventQueue changeEventQueue; + + public MongodbFetchTaskContext( + MongodbDialect dialect, + MongodbSourceConfig sourceConfig, + ChangeStreamDescriptor changeStreamDescriptor) { + this.dialect = dialect; + this.sourceConfig = sourceConfig; + this.changeStreamDescriptor = changeStreamDescriptor; + } + + public void configure(@Nonnull SourceSplitBase sourceSplitBase) { + final int queueSize = + sourceSplitBase.isSnapshotSplit() ? Integer.MAX_VALUE : sourceConfig.getBatchSize(); + this.changeEventQueue = + new ChangeEventQueue.Builder() + .pollInterval(Duration.ofMillis(sourceConfig.getPollAwaitTimeMillis())) + .maxBatchSize(sourceConfig.getPollMaxBatchSize()) + .maxQueueSize(queueSize) + .loggingContextSupplier( + () -> + LoggingContext.forConnector( + "mongodb-cdc", + "mongodb-cdc-connector", + "mongodb-cdc-connector-task")) + .build(); + } + + public MongodbSourceConfig getSourceConfig() { + return sourceConfig; + } + + public MongodbDialect getDialect() { + return dialect; + } + + public ChangeStreamDescriptor getChangeStreamDescriptor() { + return changeStreamDescriptor; + } + + public ChangeEventQueue getQueue() { + return changeEventQueue; + } + + @Override + public TableId getTableId(SourceRecord record) { + return MongodbRecordUtils.getTableId(record); + } + + @Override + public Tables.TableFilter getTableFilter() { + // We have pushed down the filters to server side. + return Tables.TableFilter.includeAll(); + } + + @Override + public boolean isExactlyOnce() { + return true; + } + + @Override + public Offset getStreamOffset(SourceRecord record) { + return new ChangeStreamOffset(getResumeToken(record)); + } + + @Override + public boolean isDataChangeRecord(SourceRecord record) { + return MongodbRecordUtils.isDataChangeRecord(record); + } + + @Override + public boolean isRecordBetween( + SourceRecord record, @Nonnull Object[] splitStart, @Nonnull Object[] splitEnd) { + BsonDocument documentKey = getDocumentKey(record); + BsonDocument splitKeys = (BsonDocument) ((Object[]) splitStart[0])[0]; + String firstKey = splitKeys.getFirstKey(); + BsonValue keyValue = documentKey.get(firstKey); + BsonValue lowerBound = ((BsonDocument) ((Object[]) splitEnd[0])[1]).get(firstKey); + BsonValue upperBound = ((BsonDocument) ((Object[]) splitEnd[0])[1]).get(firstKey); + + if (isFullRange(lowerBound, upperBound)) { + return true; + } + + return isValueInRange(lowerBound, keyValue, upperBound); + } + + private boolean isFullRange(@Nonnull BsonValue lowerBound, BsonValue upperBound) { + return lowerBound.getBsonType() == BsonType.MIN_KEY + && upperBound.getBsonType() == BsonType.MAX_KEY; + } + + private boolean isValueInRange(BsonValue lowerBound, BsonValue value, BsonValue upperBound) { + return compareBsonValue(lowerBound, value) <= 0 && compareBsonValue(value, upperBound) < 0; + } + + @Override + public void rewriteOutputBuffer( + Map outputBuffer, @Nonnull SourceRecord changeRecord) { + Struct key = (Struct) changeRecord.key(); + Struct value = (Struct) changeRecord.value(); + + if (value != null) { + String operationType = value.getString(OPERATION_TYPE); + + switch (OperationType.fromString(operationType)) { + case INSERT: + case UPDATE: + case REPLACE: + outputBuffer.put(key, changeRecord); + break; + case DELETE: + outputBuffer.remove(key); + break; + default: + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Data change record meet UNKNOWN operation: " + operationType); + } + } + } + + @Override + public List formatMessageTimestamp( + @Nonnull Collection snapshotRecords) { + return snapshotRecords.stream() + .peek( + record -> { + Struct value = (Struct) record.value(); + Struct source = new Struct(value.schema().field(SOURCE_FIELD).schema()); + source.put(TS_MS_FIELD, 0L); + source.put(SNAPSHOT_FIELD, SNAPSHOT_TRUE); + value.put(SOURCE_FIELD, source); + }) + .collect(Collectors.toList()); + } + + @Override + public void close() { + Runtime.getRuntime() + .addShutdownHook(new Thread(() -> createMongoClient(sourceConfig).close())); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbScanFetchTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbScanFetchTask.java new file mode 100644 index 000000000000..d59ecfe3646f --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbScanFetchTask.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch; + +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkEvent; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkKind; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.dialect.MongodbDialect; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils; + +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonInt64; +import org.bson.BsonString; +import org.bson.RawBsonDocument; + +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.pipeline.DataChangeEvent; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.Collections; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.COLL_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DB_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.FULL_DOCUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.OPERATION_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.OPERATION_TYPE_INSERT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_TRUE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SOURCE_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.TS_MS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createSourceOffsetMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createWatermarkPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.createMongoClient; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getMongoCollection; + +@Slf4j +public class MongodbScanFetchTask implements FetchTask { + + private final SnapshotSplit snapshotSplit; + + private volatile boolean taskRunning = false; + + public MongodbScanFetchTask(SnapshotSplit snapshotSplit) { + this.snapshotSplit = snapshotSplit; + } + + @Override + public void execute(Context context) throws Exception { + MongodbFetchTaskContext taskContext = (MongodbFetchTaskContext) context; + MongodbSourceConfig sourceConfig = taskContext.getSourceConfig(); + MongodbDialect dialect = taskContext.getDialect(); + ChangeEventQueue changeEventQueue = taskContext.getQueue(); + taskRunning = true; + TableId collectionId = snapshotSplit.getTableId(); + final ChangeStreamOffset lowWatermark = dialect.displayCurrentOffset(sourceConfig); + log.info( + "Snapshot step 1 - Determining low watermark {} for split {}", + lowWatermark, + snapshotSplit); + changeEventQueue.enqueue( + new DataChangeEvent( + WatermarkEvent.create( + createWatermarkPartitionMap(collectionId.identifier()), + "__mongodb_watermarks", + snapshotSplit.splitId(), + WatermarkKind.LOW, + lowWatermark))); + + log.info("Snapshot step 2 - Snapshotting data"); + try (MongoCursor cursor = getSnapshotCursor(snapshotSplit, sourceConfig)) { + while (cursor.hasNext()) { + checkTaskRunning(); + BsonDocument valueDocument = normalizeSnapshotDocument(collectionId, cursor.next()); + BsonDocument keyDocument = new BsonDocument(ID_FIELD, valueDocument.get(ID_FIELD)); + + SourceRecord snapshotRecord = + buildSourceRecord(sourceConfig, collectionId, keyDocument, valueDocument); + + changeEventQueue.enqueue(new DataChangeEvent(snapshotRecord)); + } + + ChangeStreamOffset highWatermark = dialect.displayCurrentOffset(sourceConfig); + log.info( + "Snapshot step 3 - Determining high watermark {} for split {}", + highWatermark, + snapshotSplit); + changeEventQueue.enqueue( + new DataChangeEvent( + WatermarkEvent.create( + createWatermarkPartitionMap(collectionId.identifier()), + "__mongodb_watermarks", + snapshotSplit.splitId(), + WatermarkKind.HIGH, + highWatermark))); + + log.info( + "Snapshot step 4 - Back fill stream split for snapshot split {}", + snapshotSplit); + final IncrementalSplit dataBackfillSplit = + createBackfillStreamSplit(lowWatermark, highWatermark); + final boolean streamBackfillRequired = + dataBackfillSplit.getStopOffset().isAfter(dataBackfillSplit.getStartupOffset()); + + if (!streamBackfillRequired) { + changeEventQueue.enqueue( + new DataChangeEvent( + WatermarkEvent.create( + createWatermarkPartitionMap(collectionId.identifier()), + "__mongodb_watermarks", + dataBackfillSplit.splitId(), + WatermarkKind.END, + dataBackfillSplit.getStopOffset()))); + } else { + MongodbStreamFetchTask dataBackfillTask = + new MongodbStreamFetchTask(dataBackfillSplit); + dataBackfillTask.execute(taskContext); + } + } catch (Exception e) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + String.format( + "Execute snapshot read subtask for mongodb split %s fail", + snapshotSplit)); + } finally { + taskRunning = false; + } + } + + @Nonnull + private MongoCursor getSnapshotCursor( + @Nonnull SnapshotSplit snapshotSplit, MongodbSourceConfig sourceConfig) { + MongoClient mongoClient = createMongoClient(sourceConfig); + MongoCollection collection = + getMongoCollection(mongoClient, snapshotSplit.getTableId(), RawBsonDocument.class); + BsonDocument startKey = (BsonDocument) snapshotSplit.getSplitStart()[1]; + BsonDocument endKey = (BsonDocument) snapshotSplit.getSplitEnd()[1]; + BsonDocument hint = (BsonDocument) snapshotSplit.getSplitStart()[0]; + log.info( + "Initializing snapshot split processing: TableId={}, StartKey={}, EndKey={}, Hint={}", + snapshotSplit.getTableId(), + startKey, + endKey, + hint); + return collection + .find() + .min(startKey) + .max(endKey) + .hint(hint) + .batchSize(sourceConfig.getBatchSize()) + .noCursorTimeout(true) + .cursor(); + } + + @Nonnull + private SourceRecord buildSourceRecord( + @Nonnull MongodbSourceConfig sourceConfig, + @Nonnull TableId collectionId, + BsonDocument keyDocument, + BsonDocument valueDocument) { + return MongodbRecordUtils.buildSourceRecord( + createPartitionMap( + sourceConfig.getHosts(), collectionId.catalog(), collectionId.table()), + createSourceOffsetMap(keyDocument.getDocument(ID_FIELD), true), + collectionId.identifier(), + keyDocument, + valueDocument); + } + + private void checkTaskRunning() { + if (!taskRunning) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Interrupted while snapshotting collection"); + } + } + + @Override + public boolean isRunning() { + return taskRunning; + } + + @Override + public void shutdown() { + taskRunning = false; + } + + @Override + public SnapshotSplit getSplit() { + return snapshotSplit; + } + + private IncrementalSplit createBackfillStreamSplit( + ChangeStreamOffset lowWatermark, ChangeStreamOffset highWatermark) { + return new IncrementalSplit( + snapshotSplit.splitId(), + Collections.singletonList(snapshotSplit.getTableId()), + lowWatermark, + highWatermark, + new ArrayList<>()); + } + + private BsonDocument normalizeSnapshotDocument( + @Nonnull final TableId collectionId, @Nonnull final BsonDocument originalDocument) { + return new BsonDocument() + .append(ID_FIELD, new BsonDocument(ID_FIELD, originalDocument.get(ID_FIELD))) + .append(OPERATION_TYPE, new BsonString(OPERATION_TYPE_INSERT)) + .append( + NS_FIELD, + new BsonDocument(DB_FIELD, new BsonString(collectionId.catalog())) + .append(COLL_FIELD, new BsonString(collectionId.table()))) + .append(DOCUMENT_KEY, new BsonDocument(ID_FIELD, originalDocument.get(ID_FIELD))) + .append(FULL_DOCUMENT, originalDocument) + .append(TS_MS_FIELD, new BsonInt64(System.currentTimeMillis())) + .append( + SOURCE_FIELD, + new BsonDocument(SNAPSHOT_FIELD, new BsonString(SNAPSHOT_TRUE)) + .append(TS_MS_FIELD, new BsonInt64(0L))); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbStreamFetchTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbStreamFetchTask.java new file mode 100644 index 000000000000..c907aa102cc4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbStreamFetchTask.java @@ -0,0 +1,359 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch; + +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkEvent; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkKind; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamDescriptor; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils; + +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonInt64; +import org.bson.BsonString; +import org.bson.BsonTimestamp; +import org.bson.Document; + +import com.mongodb.MongoCommandException; +import com.mongodb.MongoNamespace; +import com.mongodb.client.ChangeStreamIterable; +import com.mongodb.client.MongoChangeStreamCursor; +import com.mongodb.client.MongoClient; +import com.mongodb.kafka.connect.source.heartbeat.HeartbeatManager; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.pipeline.DataChangeEvent; +import lombok.extern.slf4j.Slf4j; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; + +import java.time.Instant; +import java.util.Optional; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_OPERATION; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.CLUSTER_TIME_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.COLL_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DB_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.FAILED_TO_PARSE_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.FALSE_FALSE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ILLEGAL_OPERATION_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SOURCE_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.TS_MS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UNAUTHORIZED_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UNKNOWN_FIELD_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset.NO_STOPPING_OFFSET; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createHeartbeatPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createSourceOffsetMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createWatermarkPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.currentBsonTimestamp; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.getResumeToken; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.createMongoClient; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getChangeStreamIterable; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getCurrentClusterTime; + +@Slf4j +public class MongodbStreamFetchTask implements FetchTask { + + private final IncrementalSplit streamSplit; + private volatile boolean taskRunning = false; + + private MongodbSourceConfig sourceConfig; + private final Time time = new SystemTime(); + private boolean supportsStartAtOperationTime = true; + private boolean supportsStartAfter = true; + + public MongodbStreamFetchTask(IncrementalSplit streamSplit) { + this.streamSplit = streamSplit; + } + + @Override + public void execute(Context context) { + MongodbFetchTaskContext taskContext = (MongodbFetchTaskContext) context; + this.sourceConfig = taskContext.getSourceConfig(); + + ChangeStreamDescriptor descriptor = taskContext.getChangeStreamDescriptor(); + ChangeEventQueue queue = taskContext.getQueue(); + + MongoClient mongoClient = createMongoClient(sourceConfig); + MongoChangeStreamCursor changeStreamCursor = + openChangeStreamCursor(descriptor); + HeartbeatManager heartbeatManager = openHeartbeatManagerIfNeeded(changeStreamCursor); + + final long startPoll = time.milliseconds(); + long nextUpdate = startPoll + sourceConfig.getPollAwaitTimeMillis(); + this.taskRunning = true; + try { + while (taskRunning) { + Optional next = Optional.ofNullable(changeStreamCursor.tryNext()); + SourceRecord changeRecord = null; + if (!next.isPresent()) { + long untilNext = nextUpdate - time.milliseconds(); + if (untilNext > 0) { + log.debug("Waiting {} ms to poll change records", untilNext); + time.sleep(untilNext); + continue; + } + + if (heartbeatManager != null) { + changeRecord = + heartbeatManager + .heartbeat() + .map(this::normalizeHeartbeatRecord) + .orElse(null); + } + // update nextUpdateTime + nextUpdate = time.milliseconds() + sourceConfig.getPollAwaitTimeMillis(); + } else { + BsonDocument changeStreamDocument = next.get(); + MongoNamespace namespace = getMongoNamespace(changeStreamDocument); + + BsonDocument resumeToken = changeStreamDocument.getDocument(ID_FIELD); + BsonDocument valueDocument = + normalizeChangeStreamDocument(changeStreamDocument); + + log.trace("Adding {} to {}", valueDocument, namespace.getFullName()); + + changeRecord = + MongodbRecordUtils.buildSourceRecord( + createPartitionMap( + sourceConfig.getHosts(), + namespace.getDatabaseName(), + namespace.getCollectionName()), + createSourceOffsetMap(resumeToken, false), + namespace.getFullName(), + changeStreamDocument.getDocument(ID_FIELD), + valueDocument); + } + + if (changeRecord != null) { + queue.enqueue(new DataChangeEvent(changeRecord)); + } + + if (isBoundedRead()) { + ChangeStreamOffset currentOffset; + if (changeRecord != null) { + currentOffset = new ChangeStreamOffset(getResumeToken(changeRecord)); + } else { + // Heartbeat is not turned on or there is no update event + currentOffset = new ChangeStreamOffset(getCurrentClusterTime(mongoClient)); + } + + // Reach the high watermark, the binlog fetcher should be finished + if (currentOffset.isAtOrAfter(streamSplit.getStopOffset())) { + // send watermark end event + SourceRecord watermark = + WatermarkEvent.create( + createWatermarkPartitionMap(descriptor.toString()), + "__mongodb_watermarks", + streamSplit.splitId(), + WatermarkKind.END, + currentOffset); + + queue.enqueue(new DataChangeEvent(watermark)); + break; + } + } + } + } catch (Exception e) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Poll change stream records failed"); + } finally { + taskRunning = false; + if (changeStreamCursor != null) { + changeStreamCursor.close(); + } + } + } + + @Override + public boolean isRunning() { + return taskRunning; + } + + @Override + public void shutdown() { + taskRunning = false; + } + + @Override + public IncrementalSplit getSplit() { + return streamSplit; + } + + private MongoChangeStreamCursor openChangeStreamCursor( + ChangeStreamDescriptor changeStreamDescriptor) { + ChangeStreamOffset offset = + new ChangeStreamOffset(streamSplit.getStartupOffset().getOffset()); + + ChangeStreamIterable changeStreamIterable = + getChangeStreamIterable(sourceConfig, changeStreamDescriptor); + + BsonDocument resumeToken = offset.getResumeToken(); + BsonTimestamp timestamp = offset.getTimestamp(); + + if (resumeToken != null) { + if (supportsStartAfter) { + log.info("Open the change stream after the previous offset: {}", resumeToken); + changeStreamIterable.startAfter(resumeToken); + } else { + log.info( + "Open the change stream after the previous offset using resumeAfter: {}", + resumeToken); + changeStreamIterable.resumeAfter(resumeToken); + } + } else { + if (supportsStartAtOperationTime) { + log.info("Open the change stream at the timestamp: {}", timestamp); + changeStreamIterable.startAtOperationTime(timestamp); + } else { + log.warn("Open the change stream of the latest offset"); + } + } + + try { + return (MongoChangeStreamCursor) + changeStreamIterable.withDocumentClass(BsonDocument.class).cursor(); + } catch (MongoCommandException e) { + if (e.getErrorCode() == FAILED_TO_PARSE_ERROR + || e.getErrorCode() == UNKNOWN_FIELD_ERROR) { + if (e.getErrorMessage().contains("startAtOperationTime")) { + supportsStartAtOperationTime = false; + return openChangeStreamCursor(changeStreamDescriptor); + } else if (e.getErrorMessage().contains("startAfter")) { + supportsStartAfter = false; + return openChangeStreamCursor(changeStreamDescriptor); + } else { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Open change stream failed"); + } + } else if (e.getErrorCode() == ILLEGAL_OPERATION_ERROR) { + throw new MongodbConnectorException( + UNSUPPORTED_OPERATION, + String.format( + "Illegal $changeStream operation: %s %s", + e.getErrorMessage(), e.getErrorCode())); + + } else if (e.getErrorCode() == UNAUTHORIZED_ERROR) { + throw new MongodbConnectorException( + UNSUPPORTED_OPERATION, + String.format( + "Unauthorized $changeStream operation: %s %s", + e.getErrorMessage(), e.getErrorCode())); + + } else { + throw new MongodbConnectorException(ILLEGAL_ARGUMENT, "Open change stream failed"); + } + } + } + + @Nullable private HeartbeatManager openHeartbeatManagerIfNeeded( + MongoChangeStreamCursor changeStreamCursor) { + if (sourceConfig.getHeartbeatIntervalMillis() > 0) { + return new HeartbeatManager( + time, + changeStreamCursor, + sourceConfig.getHeartbeatIntervalMillis(), + "__mongodb_heartbeats", + createHeartbeatPartitionMap(sourceConfig.getHosts())); + } + return null; + } + + @Nonnull + private BsonDocument normalizeChangeStreamDocument(@Nonnull BsonDocument changeStreamDocument) { + // _id: primary key of change document. + BsonDocument normalizedDocument = normalizeKeyDocument(changeStreamDocument); + changeStreamDocument.put(ID_FIELD, normalizedDocument); + + // ts_ms: It indicates the time at which the reader processed the event. + changeStreamDocument.put(TS_MS_FIELD, new BsonInt64(System.currentTimeMillis())); + + // source + BsonDocument source = new BsonDocument(); + source.put(SNAPSHOT_FIELD, new BsonString(FALSE_FALSE)); + + if (!changeStreamDocument.containsKey(CLUSTER_TIME_FIELD)) { + log.warn( + "Cannot extract clusterTime from change stream event, fallback to current timestamp."); + changeStreamDocument.put(CLUSTER_TIME_FIELD, currentBsonTimestamp()); + } + + // source.ts_ms + // It indicates the time that the change was made in the database. If the record is read + // from snapshot of the table instead of the change stream, the value is always 0. + BsonTimestamp clusterTime = changeStreamDocument.getTimestamp(CLUSTER_TIME_FIELD); + Instant clusterInstant = Instant.ofEpochSecond(clusterTime.getTime()); + source.put(TS_MS_FIELD, new BsonInt64(clusterInstant.toEpochMilli())); + changeStreamDocument.put(SOURCE_FIELD, source); + + return changeStreamDocument; + } + + @Nonnull + private BsonDocument normalizeKeyDocument(@Nonnull BsonDocument changeStreamDocument) { + BsonDocument documentKey = changeStreamDocument.getDocument(DOCUMENT_KEY); + BsonDocument primaryKey = new BsonDocument(ID_FIELD, documentKey.get(ID_FIELD)); + return new BsonDocument(ID_FIELD, primaryKey); + } + + @Nonnull + private SourceRecord normalizeHeartbeatRecord(@Nonnull SourceRecord heartbeatRecord) { + final Struct heartbeatValue = + new Struct(SchemaBuilder.struct().field(TS_MS_FIELD, Schema.INT64_SCHEMA).build()); + heartbeatValue.put(TS_MS_FIELD, Instant.now().toEpochMilli()); + + return new SourceRecord( + heartbeatRecord.sourcePartition(), + heartbeatRecord.sourceOffset(), + heartbeatRecord.topic(), + heartbeatRecord.keySchema(), + heartbeatRecord.key(), + SchemaBuilder.struct().field(TS_MS_FIELD, Schema.INT64_SCHEMA).build(), + heartbeatValue); + } + + @Nonnull + private MongoNamespace getMongoNamespace(@Nonnull BsonDocument changeStreamDocument) { + BsonDocument ns = changeStreamDocument.getDocument(NS_FIELD); + + return new MongoNamespace( + ns.getString(DB_FIELD).getValue(), ns.getString(COLL_FIELD).getValue()); + } + + private boolean isBoundedRead() { + return !NO_STOPPING_OFFSET.equals(streamSplit.getStopOffset()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamDescriptor.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamDescriptor.java new file mode 100644 index 000000000000..969ec1248be9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamDescriptor.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset; + +import io.debezium.relational.TableId; +import lombok.AllArgsConstructor; +import lombok.Getter; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.util.regex.Pattern; + +@AllArgsConstructor +@Getter +public class ChangeStreamDescriptor implements Serializable { + private static final long serialVersionUID = 1L; + + private final String database; + private final String collection; + private final Pattern databaseRegex; + private final Pattern namespaceRegex; + + @Nonnull + public static ChangeStreamDescriptor collection(@Nonnull TableId collectionId) { + return collection(collectionId.catalog(), collectionId.table()); + } + + @Nonnull + public static ChangeStreamDescriptor collection(String database, String collection) { + return new ChangeStreamDescriptor(database, collection, null, null); + } + + @Nonnull + public static ChangeStreamDescriptor database(String database) { + return new ChangeStreamDescriptor(database, null, null, null); + } + + @Nonnull + public static ChangeStreamDescriptor database(String database, Pattern namespaceRegex) { + return new ChangeStreamDescriptor(database, null, null, namespaceRegex); + } + + @Nonnull + public static ChangeStreamDescriptor deployment(Pattern databaseRegex) { + return new ChangeStreamDescriptor(null, null, databaseRegex, null); + } + + @Nonnull + public static ChangeStreamDescriptor deployment(Pattern databaseRegex, Pattern namespaceRegex) { + return new ChangeStreamDescriptor(null, null, databaseRegex, namespaceRegex); + } + + @Nonnull + public static ChangeStreamDescriptor deployment() { + return new ChangeStreamDescriptor(null, null, null, null); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffset.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffset.java new file mode 100644 index 000000000000..35acf43bbace --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffset.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset; + +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; + +import org.bson.BsonDocument; +import org.bson.BsonTimestamp; + +import javax.annotation.Nullable; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.RESUME_TOKEN_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.TIMESTAMP_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.maximumBsonTimestamp; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ResumeToken.decodeTimestamp; + +public class ChangeStreamOffset extends Offset { + + private static final long serialVersionUID = 1L; + + public static final ChangeStreamOffset NO_STOPPING_OFFSET = + new ChangeStreamOffset(maximumBsonTimestamp()); + + public ChangeStreamOffset(Map offset) { + this.offset = offset; + } + + public ChangeStreamOffset(BsonDocument resumeToken) { + Objects.requireNonNull(resumeToken); + Map offsetMap = new HashMap<>(); + offsetMap.put(TIMESTAMP_FIELD, String.valueOf(decodeTimestamp(resumeToken).getValue())); + offsetMap.put(RESUME_TOKEN_FIELD, resumeToken.toJson()); + this.offset = offsetMap; + } + + public ChangeStreamOffset(BsonTimestamp timestamp) { + Objects.requireNonNull(timestamp); + Map offsetMap = new HashMap<>(); + offsetMap.put(TIMESTAMP_FIELD, String.valueOf(timestamp.getValue())); + offsetMap.put(RESUME_TOKEN_FIELD, null); + this.offset = offsetMap; + } + + @Nullable public BsonDocument getResumeToken() { + String resumeTokenJson = offset.get(RESUME_TOKEN_FIELD); + return Optional.ofNullable(resumeTokenJson).map(BsonDocument::parse).orElse(null); + } + + public BsonTimestamp getTimestamp() { + long timestamp = Long.parseLong(offset.get(TIMESTAMP_FIELD)); + return new BsonTimestamp(timestamp); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ChangeStreamOffset)) { + return false; + } + ChangeStreamOffset that = (ChangeStreamOffset) o; + return offset.equals(that.offset); + } + + @Override + public int compareTo(Offset offset) { + if (offset == null) { + return -1; + } + ChangeStreamOffset that = (ChangeStreamOffset) offset; + return this.getTimestamp().compareTo(that.getTimestamp()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffsetFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffsetFactory.java new file mode 100644 index 000000000000..c53d92aa67b9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffsetFactory.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset; + +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; +import org.apache.seatunnel.connectors.cdc.base.source.offset.OffsetFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import java.util.Map; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_OPERATION; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.bsonTimestampFromEpochMillis; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.currentBsonTimestamp; + +public class ChangeStreamOffsetFactory extends OffsetFactory { + + @Override + public Offset earliest() { + return new ChangeStreamOffset(currentBsonTimestamp()); + } + + @Override + public Offset neverStop() { + return ChangeStreamOffset.NO_STOPPING_OFFSET; + } + + @Override + public Offset latest() { + return new ChangeStreamOffset(currentBsonTimestamp()); + } + + @Override + public Offset specific(Map offset) { + return new ChangeStreamOffset(offset); + } + + @Override + public Offset specific(String filename, Long position) { + throw new MongodbConnectorException( + UNSUPPORTED_OPERATION, "not supported create new Offset by filename and position."); + } + + @Override + public Offset timestamp(long timestamp) { + return new ChangeStreamOffset(bsonTimestampFromEpochMillis(timestamp)); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/MongodbChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/MongodbChunkSplitter.java new file mode 100644 index 000000000000..18549c3f4b87 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/MongodbChunkSplitter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; + +import io.debezium.relational.TableId; + +import java.util.Collection; + +public class MongodbChunkSplitter implements ChunkSplitter { + + private final MongodbSourceConfig sourceConfig; + + public MongodbChunkSplitter(MongodbSourceConfig sourceConfig) { + this.sourceConfig = sourceConfig; + } + + @Override + public Collection generateSplits(TableId collectionId) { + SplitContext splitContext = SplitContext.of(sourceConfig, collectionId); + SplitStrategy splitStrategy = + splitContext.isShardedCollection() + ? ShardedSplitStrategy.INSTANCE + : SplitVectorSplitStrategy.INSTANCE; + return splitStrategy.split(splitContext); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SampleBucketSplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SampleBucketSplitStrategy.java new file mode 100644 index 000000000000..1fb251aba002 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SampleBucketSplitStrategy.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.bson.BsonDocument; +import org.bson.BsonValue; +import org.bson.conversions.Bson; + +import com.mongodb.client.MongoCollection; +import io.debezium.relational.TableId; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import static com.mongodb.client.model.Aggregates.bucketAuto; +import static com.mongodb.client.model.Aggregates.sample; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MAX_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MIN_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.boundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.maxUpperBoundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.minLowerBoundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getMongoCollection; + +public enum SampleBucketSplitStrategy implements SplitStrategy { + INSTANCE; + + private static final int DEFAULT_SAMPLING_THRESHOLD = 102400; + + private static final double DEFAULT_SAMPLING_RATE = 0.05; + + @Nonnull + @Override + public Collection split(@Nonnull SplitContext splitContext) { + long chunkSizeInBytes = (long) splitContext.getChunkSizeMB() * 1024 * 1024; + + long sizeInBytes = splitContext.getSizeInBytes(); + long count = splitContext.getDocumentCount(); + + // If collection's total uncompressed size less than chunk size, + // treat the entire collection as single chunk. + if (sizeInBytes < chunkSizeInBytes) { + return SingleSplitStrategy.INSTANCE.split(splitContext); + } + + int numChunks = (int) (sizeInBytes / chunkSizeInBytes) + 1; + int numberOfSamples; + if (count < DEFAULT_SAMPLING_THRESHOLD) { + // full sampling if document count less than sampling size threshold. + numberOfSamples = (int) count; + } else { + // sampled using sample rate. + numberOfSamples = (int) Math.floor(count * DEFAULT_SAMPLING_RATE); + } + + TableId collectionId = splitContext.getCollectionId(); + + MongoCollection collection = + getMongoCollection(splitContext.getMongoClient(), collectionId, BsonDocument.class); + + List pipeline = new ArrayList<>(); + if (numberOfSamples != count) { + pipeline.add(sample(numberOfSamples)); + } + pipeline.add(bucketAuto("$" + ID_FIELD, numChunks)); + + List chunks = + collection.aggregate(pipeline).allowDiskUse(true).into(new ArrayList<>()); + + SeaTunnelRowType rowType = shardKeysToRowType(Collections.singleton(ID_FIELD)); + + List snapshotSplits = new ArrayList<>(chunks.size() + 2); + + SnapshotSplit firstSplit = + new SnapshotSplit( + splitId(collectionId, 0), + collectionId, + rowType, + minLowerBoundOfId(), + boundOfId(lowerBoundOfBucket(chunks.get(0)))); + snapshotSplits.add(firstSplit); + + for (int i = 0; i < chunks.size(); i++) { + BsonDocument bucket = chunks.get(i); + snapshotSplits.add( + new SnapshotSplit( + splitId(collectionId, i + 1), + collectionId, + rowType, + boundOfId(lowerBoundOfBucket(bucket)), + boundOfId(upperBoundOfBucket(bucket)))); + } + + SnapshotSplit lastSplit = + new SnapshotSplit( + splitId(collectionId, chunks.size() + 1), + collectionId, + rowType, + boundOfId(upperBoundOfBucket(chunks.get(chunks.size() - 1))), + maxUpperBoundOfId()); + snapshotSplits.add(lastSplit); + + return snapshotSplits; + } + + private BsonDocument bucketBounds(@Nonnull BsonDocument bucket) { + return bucket.getDocument(ID_FIELD); + } + + private BsonValue lowerBoundOfBucket(BsonDocument bucket) { + return bucketBounds(bucket).get(MIN_FIELD); + } + + private BsonValue upperBoundOfBucket(BsonDocument bucket) { + return bucketBounds(bucket).get(MAX_FIELD); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/ShardedSplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/ShardedSplitStrategy.java new file mode 100644 index 000000000000..8a6d3aee4bbe --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/ShardedSplitStrategy.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.bson.BsonBoolean; +import org.bson.BsonDocument; + +import com.mongodb.MongoQueryException; +import com.mongodb.client.MongoClient; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DROPPED_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MAX_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MIN_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UNAUTHORIZED_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.readChunks; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.readCollectionMetadata; + +@Slf4j +public class ShardedSplitStrategy implements SplitStrategy { + + public static final ShardedSplitStrategy INSTANCE = new ShardedSplitStrategy(); + + private ShardedSplitStrategy() {} + + @Override + public Collection split(@Nonnull SplitContext splitContext) { + TableId collectionId = splitContext.getCollectionId(); + MongoClient mongoClient = splitContext.getMongoClient(); + + List chunks; + BsonDocument collectionMetadata; + try { + collectionMetadata = readCollectionMetadata(mongoClient, collectionId); + if (!isValidShardedCollection(collectionMetadata)) { + log.warn( + "Collection {} does not appear to be sharded, fallback to SampleSplitter.", + collectionId); + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + chunks = readChunks(mongoClient, collectionMetadata); + } catch (MongoQueryException e) { + if (e.getErrorCode() == UNAUTHORIZED_ERROR) { + log.warn( + "Unauthorized to read config.collections or config.chunks: {}, fallback to SampleSplitter.", + e.getErrorMessage()); + } else { + log.warn( + "Read config.chunks collection failed: {}, fallback to SampleSplitter", + e.getErrorMessage()); + } + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + + if (chunks.isEmpty()) { + log.warn( + "Collection {} does not appear to be sharded, fallback to SampleSplitter.", + collectionId); + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + + BsonDocument splitKeys = collectionMetadata.getDocument("key"); + SeaTunnelRowType rowType = shardKeysToRowType(splitKeys); + + List snapshotSplits = new ArrayList<>(chunks.size()); + for (int i = 0; i < chunks.size(); i++) { + BsonDocument chunk = chunks.get(i); + snapshotSplits.add( + new SnapshotSplit( + splitId(collectionId, i), + collectionId, + rowType, + new Object[] {splitKeys, chunk.getDocument(MIN_FIELD)}, + new Object[] {splitKeys, chunk.getDocument(MAX_FIELD)})); + } + return snapshotSplits; + } + + private boolean isValidShardedCollection(BsonDocument collectionMetadata) { + return collectionMetadata != null + && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue(); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SingleSplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SingleSplitStrategy.java new file mode 100644 index 000000000000..c84b3e5242f5 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SingleSplitStrategy.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import io.debezium.relational.TableId; + +import javax.annotation.Nonnull; + +import java.util.Collection; +import java.util.Collections; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.maxUpperBoundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.minLowerBoundOfId; + +public enum SingleSplitStrategy implements SplitStrategy { + INSTANCE; + + @Override + public Collection split(@Nonnull SplitContext splitContext) { + TableId collectionId = splitContext.getCollectionId(); + SnapshotSplit snapshotSplit = createSnapshotSplit(collectionId); + return Collections.singletonList(snapshotSplit); + } + + @Nonnull + private SnapshotSplit createSnapshotSplit(TableId collectionId) { + SeaTunnelRowType rowType = shardKeysToRowType(Collections.singleton(ID_FIELD)); + return new SnapshotSplit( + splitId(collectionId, 0), + collectionId, + rowType, + minLowerBoundOfId(), + maxUpperBoundOfId()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitContext.java new file mode 100644 index 000000000000..0af67673f9a3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitContext.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils; + +import org.bson.BsonBoolean; +import org.bson.BsonDocument; +import org.bson.BsonInt64; +import org.bson.BsonNumber; + +import com.mongodb.client.MongoClient; +import io.debezium.relational.TableId; + +import javax.annotation.Nonnull; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.collStats; + +public class SplitContext { + + private final MongoClient mongoClient; + private final TableId collectionId; + private final BsonDocument collectionStats; + private final int chunkSizeMB; + + public SplitContext( + MongoClient mongoClient, + TableId collectionId, + BsonDocument collectionStats, + int chunkSizeMB) { + this.mongoClient = mongoClient; + this.collectionId = collectionId; + this.collectionStats = collectionStats; + this.chunkSizeMB = chunkSizeMB; + } + + @Nonnull + public static SplitContext of(MongodbSourceConfig sourceConfig, TableId collectionId) { + MongoClient mongoClient = MongodbUtils.createMongoClient(sourceConfig); + BsonDocument collectionStats = collStats(mongoClient, collectionId); + int chunkSizeMB = sourceConfig.getSplitSize(); + return new SplitContext(mongoClient, collectionId, collectionStats, chunkSizeMB); + } + + public MongoClient getMongoClient() { + return mongoClient; + } + + public TableId getCollectionId() { + return collectionId; + } + + public int getChunkSizeMB() { + return chunkSizeMB; + } + + public long getDocumentCount() { + return getNumberValue(collectionStats, "count"); + } + + public long getSizeInBytes() { + return getNumberValue(collectionStats, "size"); + } + + public long getAvgObjSizeInBytes() { + return getNumberValue(collectionStats, "avgObjSize"); + } + + public boolean isShardedCollection() { + return collectionStats.getBoolean("sharded", BsonBoolean.FALSE).getValue(); + } + + private long getNumberValue(@Nonnull BsonDocument document, String fieldName) { + BsonNumber number = document.getNumber(fieldName, new BsonInt64(0)); + return number.longValue(); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitStrategy.java new file mode 100644 index 000000000000..6e3f4aa34eba --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitStrategy.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.bson.BsonDocument; + +import io.debezium.relational.TableId; + +import javax.annotation.Nonnull; + +import java.util.Collection; + +import static org.apache.seatunnel.api.table.type.BasicType.INT_TYPE; + +public interface SplitStrategy { + + Collection split(SplitContext splitContext); + + default String splitId(@Nonnull TableId collectionId, int chunkId) { + return String.format("%s:%d", collectionId.identifier(), chunkId); + } + + default SeaTunnelRowType shardKeysToRowType(@Nonnull BsonDocument shardKeys) { + return shardKeysToRowType(shardKeys.keySet()); + } + + default SeaTunnelRowType shardKeysToRowType(@Nonnull Collection shardKeys) { + SeaTunnelDataType[] fieldTypes = + shardKeys.stream() + // We cannot get the exact type of the shard key, only the ordering of the + // shard index. + // Use the INT type as a placeholder. + .map(key -> INT_TYPE) + .toArray(SeaTunnelDataType[]::new); + String[] fieldNames = shardKeys.toArray(new String[0]); + return new SeaTunnelRowType(fieldNames, fieldTypes); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitVectorSplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitVectorSplitStrategy.java new file mode 100644 index 000000000000..1421c5123eab --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitVectorSplitStrategy.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.apache.commons.collections4.CollectionUtils; + +import org.bson.BsonArray; +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonMinKey; +import org.bson.BsonValue; + +import com.mongodb.MongoCommandException; +import com.mongodb.client.MongoClient; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import javax.annotation.Nonnull; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UNAUTHORIZED_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.boundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.maxUpperBoundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.isCommandSucceed; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.splitVector; + +@Slf4j +public enum SplitVectorSplitStrategy implements SplitStrategy { + INSTANCE; + + @Override + public Collection split(@Nonnull SplitContext splitContext) { + MongoClient mongoClient = splitContext.getMongoClient(); + TableId collectionId = splitContext.getCollectionId(); + int chunkSizeMB = splitContext.getChunkSizeMB(); + + BsonDocument keyPattern = new BsonDocument(ID_FIELD, new BsonInt32(1)); + + BsonDocument splitResult; + try { + splitResult = splitVector(mongoClient, collectionId, keyPattern, chunkSizeMB); + } catch (MongoCommandException e) { + if (e.getErrorCode() == UNAUTHORIZED_ERROR) { + log.warn( + "Unauthorized to execute splitVector command: {}, fallback to SampleSplitter", + e.getErrorMessage()); + } else { + log.warn( + "Execute splitVector command failed: {}, fallback to SampleSplitter", + e.getErrorMessage()); + } + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + + if (!isCommandSucceed(splitResult)) { + log.warn( + "Could not calculate standalone splits: {}, fallback to SampleSplitter", + splitResult.getString("errmsg")); + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + + BsonArray splitKeys = splitResult.getArray("splitKeys"); + if (CollectionUtils.isEmpty(splitKeys)) { + // documents size is less than chunk size, treat the entire collection as single chunk. + return SingleSplitStrategy.INSTANCE.split(splitContext); + } + + SeaTunnelRowType rowType = shardKeysToRowType(Collections.singleton(ID_FIELD)); + List snapshotSplits = new ArrayList<>(splitKeys.size() + 1); + + BsonValue lowerValue = new BsonMinKey(); + ; + for (int i = 0; i < splitKeys.size(); i++) { + BsonValue splitKeyValue = splitKeys.get(i).asDocument().get(ID_FIELD); + snapshotSplits.add( + new SnapshotSplit( + splitId(collectionId, i), + collectionId, + rowType, + boundOfId(lowerValue), + boundOfId(splitKeyValue))); + lowerValue = splitKeyValue; + } + + SnapshotSplit lastSplit = + new SnapshotSplit( + splitId(collectionId, splitKeys.size()), + collectionId, + rowType, + boundOfId(lowerValue), + maxUpperBoundOfId()); + snapshotSplits.add(lastSplit); + + return snapshotSplits; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/BsonUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/BsonUtils.java new file mode 100644 index 000000000000..4c46e36b708c --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/BsonUtils.java @@ -0,0 +1,325 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import org.bson.BsonArray; +import org.bson.BsonBinary; +import org.bson.BsonDbPointer; +import org.bson.BsonDocument; +import org.bson.BsonJavaScriptWithScope; +import org.bson.BsonNumber; +import org.bson.BsonObjectId; +import org.bson.BsonString; +import org.bson.BsonType; +import org.bson.BsonUndefined; +import org.bson.BsonValue; +import org.bson.types.Decimal128; + +import javax.annotation.Nonnull; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; + +public class BsonUtils { + + public static int compareBsonValue(BsonValue o1, BsonValue o2) { + return compareBsonValue(o1, o2, true); + } + + private static int compareBsonValue(BsonValue o1, BsonValue o2, boolean isTopLevel) { + if (Objects.equals(o1, o2)) { + return 0; + } + + if (isTopLevel) { + BsonValue element1 = o1; + BsonValue element2 = o2; + + if (o1 != null && o1.isArray()) { + element1 = smallestValueOfArray(o1.asArray()); + } + if (o2.isArray()) { + element2 = smallestValueOfArray(o2.asArray()); + } + return compareBsonValues(element1, element2); + } + if (typeOrder(o1) != typeOrder(o2)) { + return Integer.compare(typeOrder(o1), typeOrder(o2)); + } + + if (isNull(o1) || isMinKey(o1) || isMaxKey(o1)) { + return 0; // Null == Null, MinKey == MinKey, MaxKey == MaxKey + } + + switch (o1.getBsonType()) { + case INT32: + case INT64: + case DOUBLE: + return compareBsonNumbers(o1.asNumber(), o2.asNumber()); + case STRING: + case JAVASCRIPT: + case REGULAR_EXPRESSION: + return compareStrings(o1.asString().getValue(), o2.asString().getValue()); + case BOOLEAN: + return compareBooleans(o1.asBoolean().getValue(), o2.asBoolean().getValue()); + case DATE_TIME: + return compareDateTimes(o1.asDateTime().getValue(), o2.asDateTime().getValue()); + case TIMESTAMP: + return compareTimestamps(o1.asTimestamp().getValue(), o2.asTimestamp().getValue()); + case BINARY: + return compareBsonBinary(o1.asBinary(), o2.asBinary()); + case OBJECT_ID: + return o1.asObjectId().compareTo(o2.asObjectId()); + case DOCUMENT: + case DB_POINTER: + return compareBsonDocument(toBsonDocument(o1), toBsonDocument(o2)); + case ARRAY: + return compareBsonArray(o1.asArray(), o2.asArray()); + case JAVASCRIPT_WITH_SCOPE: + return compareJavascriptWithScope( + o1.asJavaScriptWithScope(), o2.asJavaScriptWithScope()); + default: + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + String.format("Unable to compare bson values between %s and %s", o1, o2)); + } + } + + private static int compareBsonValues(BsonValue v1, BsonValue v2) { + return compareBsonValue(v1, v2, false); + } + + private static int compareBsonNumbers(BsonNumber n1, BsonNumber n2) { + Decimal128 decimal1 = getDecimal128FromCache(n1); + Decimal128 decimal2 = getDecimal128FromCache(n2); + return decimal1.compareTo(decimal2); + } + + private static int compareStrings(String s1, String s2) { + return getStringFromCache(s1).compareTo(getStringFromCache(s2)); + } + + private static int compareBooleans(boolean b1, boolean b2) { + return Boolean.compare(b1, b2); + } + + private static int compareDateTimes(long dt1, long dt2) { + return Long.compare(dt1, dt2); + } + + private static int compareTimestamps(long ts1, long ts2) { + return Long.compare(ts1, ts2); + } + + private static final Map decimalCache = new HashMap<>(); + private static final Map stringCache = new HashMap<>(); + + private static Decimal128 getDecimal128FromCache(BsonValue value) { + return decimalCache.computeIfAbsent(value, BsonUtils::toDecimal128); + } + + private static String getStringFromCache(String value) { + return stringCache.computeIfAbsent(value, k -> k); + } + + public static int compareBsonDocument(@Nonnull BsonDocument d1, @Nonnull BsonDocument d2) { + Iterator> iterator1 = d1.entrySet().iterator(); + Iterator> iterator2 = d2.entrySet().iterator(); + + if (!iterator1.hasNext() && !iterator2.hasNext()) { + return 0; + } else if (!iterator1.hasNext()) { + return -1; + } else if (!iterator2.hasNext()) { + return 1; + } else { + while (iterator1.hasNext() && iterator2.hasNext()) { + Map.Entry entry1 = iterator1.next(); + Map.Entry entry2 = iterator2.next(); + + int result = + Integer.compare(typeOrder(entry1.getValue()), typeOrder(entry2.getValue())); + if (result != 0) { + return result; + } + + result = entry1.getKey().compareTo(entry2.getKey()); + if (result != 0) { + return result; + } + + result = compareBsonValue(entry1.getValue(), entry2.getValue(), false); + if (result != 0) { + return result; + } + } + + return Integer.compare(d1.size(), d2.size()); + } + } + + public static int compareBsonArray(BsonArray a1, BsonArray a2) { + return compareBsonValue(smallestValueOfArray(a1), smallestValueOfArray(a2), false); + } + + private static BsonValue smallestValueOfArray(@Nonnull BsonArray bsonArray) { + if (bsonArray.isEmpty()) { + return new BsonUndefined(); + } + + if (bsonArray.size() == 1) { + return bsonArray.get(0); + } + + return bsonArray.getValues().stream() + .min((e1, e2) -> compareBsonValue(e1, e2, false)) + .orElseThrow( + () -> + new IllegalStateException( + "Unable to find smallest value in the array.")); + } + + public static int compareBsonBinary(@Nonnull BsonBinary b1, @Nonnull BsonBinary b2) { + byte[] data1 = b1.getData(); + byte[] data2 = b2.getData(); + + int lengthComparison = Integer.compare(data1.length, data2.length); + if (lengthComparison != 0) { + return lengthComparison; + } + + int typeComparison = Byte.compare(b1.getType(), b2.getType()); + if (typeComparison != 0) { + return typeComparison; + } + + for (int i = 0; i < data1.length; i++) { + int byteComparison = Integer.compareUnsigned(data1[i] & 0xff, data2[i] & 0xff); + if (byteComparison != 0) { + return byteComparison; + } + } + + return 0; + } + + public static int compareJavascriptWithScope( + @Nonnull BsonJavaScriptWithScope c1, @Nonnull BsonJavaScriptWithScope c2) { + int result = c1.getCode().compareTo(c2.getCode()); + if (result != 0) { + return result; + } + return compareBsonDocument(c1.getScope(), c2.getScope()); + } + + public static boolean isNull(BsonValue bsonValue) { + return bsonValue == null + || bsonValue.isNull() + || bsonValue.getBsonType() == BsonType.UNDEFINED; + } + + public static boolean isMinKey(BsonValue bsonValue) { + return bsonValue != null && bsonValue.getBsonType() == BsonType.MIN_KEY; + } + + public static boolean isMaxKey(BsonValue bsonValue) { + return bsonValue != null && bsonValue.getBsonType() == BsonType.MAX_KEY; + } + + public static Decimal128 toDecimal128(@Nonnull BsonValue bsonValue) { + if (bsonValue.isNumber()) { + return bsonValue.asNumber().decimal128Value(); + } else if (bsonValue.isDecimal128()) { + return bsonValue.asDecimal128().decimal128Value(); + } else { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Cannot convert to Decimal128 with unexpected value: " + bsonValue); + } + } + + public static BsonDocument toBsonDocument(@Nonnull BsonValue bsonValue) { + if (bsonValue.isDocument()) { + return bsonValue.asDocument(); + } else if (bsonValue.isDBPointer()) { + BsonDbPointer dbPointer = bsonValue.asDBPointer(); + return new BsonDocument("$ref", new BsonString(dbPointer.getNamespace())) + .append("$id", new BsonObjectId(dbPointer.getId())); + } + + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Cannot convert to Document with unexpected value: " + bsonValue); + } + + public static int typeOrder(BsonValue bsonValue) { + // Missing Key field + if (bsonValue == null) { + return 3; + } + + BsonType bsonType = bsonValue.getBsonType(); + switch (bsonType) { + case MIN_KEY: + return 1; + case UNDEFINED: + return 2; + case NULL: + return 3; + case INT32: + case INT64: + case DOUBLE: + case DECIMAL128: + return 4; + case STRING: + case SYMBOL: + return 5; + case DOCUMENT: + case DB_POINTER: + return 6; + case ARRAY: + return 7; + case BINARY: + return 8; + case OBJECT_ID: + return 9; + case BOOLEAN: + return 10; + case DATE_TIME: + return 11; + case TIMESTAMP: + return 12; + case REGULAR_EXPRESSION: + return 13; + case JAVASCRIPT: + return 14; + case JAVASCRIPT_WITH_SCOPE: + return 15; + case MAX_KEY: + return 99; + default: + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Unknown bson type : " + bsonType); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ChunkUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ChunkUtils.java new file mode 100644 index 000000000000..aad0a3e42b82 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ChunkUtils.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonMaxKey; +import org.bson.BsonMinKey; +import org.bson.BsonValue; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; + +public class ChunkUtils { + + private ChunkUtils() {} + + public static Object[] boundOfId(BsonValue bound) { + return new Object[] { + new BsonDocument(ID_FIELD, new BsonInt32(1)), new BsonDocument(ID_FIELD, bound) + }; + } + + public static Object[] minLowerBoundOfId() { + return boundOfId(new BsonMinKey()); + } + + public static Object[] maxUpperBoundOfId() { + return boundOfId(new BsonMaxKey()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/CollectionDiscoveryUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/CollectionDiscoveryUtils.java new file mode 100644 index 000000000000..bef7b03e64ec --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/CollectionDiscoveryUtils.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.commons.collections4.CollectionUtils; + +import org.bson.BsonDocument; +import org.bson.conversions.Bson; + +import com.mongodb.MongoNamespace; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoDatabase; +import lombok.AllArgsConstructor; +import lombok.Getter; + +import javax.annotation.Nonnull; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ADD_NS_FIELD_NAME; + +public class CollectionDiscoveryUtils { + + public static final Bson ADD_NS_FIELD = + BsonDocument.parse( + String.format( + "{'$addFields': {'%s': {'$concat': ['$ns.db', '.', '$ns.coll']}}}", + ADD_NS_FIELD_NAME)); + + private CollectionDiscoveryUtils() {} + + public static @Nonnull List databaseNames( + @Nonnull MongoClient mongoClient, Predicate databaseFilter) { + List databaseNames = new ArrayList<>(); + return mongoClient.listDatabaseNames().into(databaseNames).stream() + .filter(databaseFilter) + .collect(Collectors.toList()); + } + + public static @Nonnull List collectionNames( + MongoClient mongoClient, + List databaseNames, + Predicate collectionFilter) { + return collectionNames(mongoClient, databaseNames, collectionFilter, String::toString); + } + + public static @Nonnull List collectionNames( + MongoClient mongoClient, + @Nonnull List databaseNames, + Predicate collectionFilter, + Function conversion) { + List collectionNames = new ArrayList<>(); + for (String dbName : databaseNames) { + MongoDatabase db = mongoClient.getDatabase(dbName); + StreamSupport.stream(db.listCollectionNames().spliterator(), false) + .map(collName -> dbName + "." + collName) + .filter(collectionFilter) + .map(conversion) + .forEach(collectionNames::add); + } + return collectionNames; + } + + private static Predicate stringListFilter( + Predicate filter, List stringList) { + if (CollectionUtils.isNotEmpty(stringList)) { + List databasePatterns = includeListAsPatterns(stringList); + filter = filter.and(anyMatch(databasePatterns)); + } + return filter; + } + + public static Predicate databaseFilter(List databaseList) { + return stringListFilter(CollectionDiscoveryUtils::isNotBuiltInDatabase, databaseList); + } + + public static Predicate collectionsFilter(List collectionList) { + return stringListFilter(CollectionDiscoveryUtils::isNotBuiltInCollections, collectionList); + } + + public static @Nonnull Predicate anyMatch(List patterns) { + return s -> patterns.stream().anyMatch(p -> p.matcher(s).matches()); + } + + public static Pattern includeListAsFlatPattern(List includeList) { + return includeListAsFlatPattern(includeList, CollectionDiscoveryUtils::completionPattern); + } + + public static Pattern includeListAsFlatPattern( + List includeList, Function conversion) { + if (includeList == null || includeList.isEmpty()) { + return null; + } + String flatPatternLiteral = + includeList.stream() + .map(conversion) + .map(Pattern::pattern) + .collect(Collectors.joining("|")); + + return Pattern.compile(flatPatternLiteral); + } + + public static List includeListAsPatterns(List includeList) { + return includeListAsPatterns(includeList, CollectionDiscoveryUtils::completionPattern); + } + + public static List includeListAsPatterns( + List includeList, Function conversion) { + return includeList != null && !includeList.isEmpty() + ? includeList.stream().map(conversion).collect(Collectors.toList()) + : Collections.emptyList(); + } + + public static boolean isNotBuiltInCollections(String fullName) { + if (fullName == null) { + return false; + } + MongoNamespace namespace = new MongoNamespace(fullName); + return isNotBuiltInDatabase(namespace.getDatabaseName()) + && !namespace.getCollectionName().startsWith("system."); + } + + public static boolean isNotBuiltInDatabase(String databaseName) { + if (databaseName == null) { + return false; + } + return !"local".equals(databaseName) + && !"admin".equals(databaseName) + && !"config".equals(databaseName); + } + + public static @Nonnull Pattern completionPattern(@Nonnull String pattern) { + if (pattern.startsWith("^") && pattern.endsWith("$")) { + return Pattern.compile(pattern); + } + return Pattern.compile("^(" + pattern + ")$"); + } + + @Getter + @AllArgsConstructor + public static class CollectionDiscoveryInfo implements Serializable { + private static final long serialVersionUID = 1L; + + private final List discoveredDatabases; + + private final List discoveredCollections; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java new file mode 100644 index 000000000000..84af2f7fda62 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.commons.lang3.StringUtils; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonTimestamp; +import org.bson.json.JsonWriterSettings; + +import com.mongodb.kafka.connect.source.json.formatter.DefaultJson; +import com.mongodb.kafka.connect.source.schema.AvroSchemaDefaults; +import com.mongodb.kafka.connect.source.schema.BsonValueToSchemaAndValue; +import io.debezium.relational.TableId; + +import javax.annotation.Nonnull; + +import java.time.Instant; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static com.mongodb.kafka.connect.source.schema.AvroSchema.fromJson; +import static org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkEvent.isWatermarkEvent; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.COLL_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DB_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.OUTPUT_SCHEMA; + +public class MongodbRecordUtils { + + public static boolean isHeartbeatEvent(SourceRecord sourceRecord) { + return "true".equals(getOffsetValue(sourceRecord, "copy")); + } + + public static boolean isDataChangeRecord(SourceRecord sourceRecord) { + return !isWatermarkEvent(sourceRecord) && !isHeartbeatEvent(sourceRecord); + } + + public static BsonDocument getResumeToken(SourceRecord sourceRecord) { + return BsonDocument.parse(getOffsetValue(sourceRecord, ID_FIELD)); + } + + public static BsonDocument getDocumentKey(@Nonnull SourceRecord sourceRecord) { + Struct value = (Struct) sourceRecord.value(); + return BsonDocument.parse(value.getString(DOCUMENT_KEY)); + } + + public static String getOffsetValue(@Nonnull SourceRecord sourceRecord, String key) { + return (String) sourceRecord.sourceOffset().get(key); + } + + public static @Nonnull TableId getTableId(@Nonnull SourceRecord dataRecord) { + Struct value = (Struct) dataRecord.value(); + Struct source = value.getStruct(NS_FIELD); + String dbName = source.getString(DB_FIELD); + String collName = source.getString(COLL_FIELD); + return new TableId(dbName, null, collName); + } + + public static @Nonnull BsonTimestamp currentBsonTimestamp() { + return bsonTimestampFromEpochMillis(System.currentTimeMillis()); + } + + public static @Nonnull BsonTimestamp maximumBsonTimestamp() { + return new BsonTimestamp(Integer.MAX_VALUE, Integer.MAX_VALUE); + } + + public static @Nonnull BsonTimestamp bsonTimestampFromEpochMillis(long epochMillis) { + return new BsonTimestamp((int) Instant.ofEpochMilli(epochMillis).getEpochSecond(), 1); + } + + public static @Nonnull SourceRecord buildSourceRecord( + final Map partition, + final Map sourceOffset, + final String topicName, + final BsonDocument keyDocument, + final BsonDocument valueDocument) { + return buildSourceRecord( + partition, + sourceOffset, + topicName, + keyDocument, + valueDocument, + new DefaultJson().getJsonWriterSettings()); + } + + public static @Nonnull SourceRecord buildSourceRecord( + Map partition, + Map sourceOffset, + String topicName, + BsonDocument keyDocument, + BsonDocument valueDocument, + JsonWriterSettings jsonWriterSettings) { + BsonValueToSchemaAndValue schemaAndValue = + new BsonValueToSchemaAndValue(jsonWriterSettings); + SchemaAndValue keySchemaAndValue = + schemaAndValue.toSchemaAndValue( + fromJson(AvroSchemaDefaults.DEFAULT_AVRO_KEY_SCHEMA), keyDocument); + SchemaAndValue valueSchemaAndValue = + schemaAndValue.toSchemaAndValue(fromJson(OUTPUT_SCHEMA), valueDocument); + + return new SourceRecord( + partition, + sourceOffset, + topicName, + keySchemaAndValue.schema(), + keySchemaAndValue.value(), + valueSchemaAndValue.schema(), + valueSchemaAndValue.value()); + } + + public static @Nonnull Map createSourceOffsetMap( + @Nonnull BsonDocument idDocument, boolean isSnapshotRecord) { + Map sourceOffset = new HashMap<>(); + sourceOffset.put(ID_FIELD, idDocument.toJson()); + sourceOffset.put("copy", String.valueOf(isSnapshotRecord)); + return sourceOffset; + } + + public static @Nonnull Map createPartitionMap( + String hosts, String database, String collection) { + StringBuilder builder = new StringBuilder(); + builder.append("mongodb://"); + builder.append(hosts); + builder.append("/"); + if (StringUtils.isNotEmpty(database)) { + builder.append(database); + } + if (StringUtils.isNotEmpty(collection)) { + builder.append("."); + builder.append(collection); + } + return Collections.singletonMap(NS_FIELD, builder.toString()); + } + + public static @Nonnull Map createHeartbeatPartitionMap(String hosts) { + String builder = "mongodb://" + hosts + "/" + "__mongodb_heartbeats"; + return Collections.singletonMap(NS_FIELD, builder); + } + + public static @Nonnull Map createWatermarkPartitionMap(String partition) { + return Collections.singletonMap(NS_FIELD, partition); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbUtils.java new file mode 100644 index 000000000000..3fd3cdb3fda4 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbUtils.java @@ -0,0 +1,407 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.internal.MongodbClientProvider; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamDescriptor; + +import org.apache.commons.lang3.StringUtils; + +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonString; +import org.bson.BsonTimestamp; +import org.bson.Document; +import org.bson.conversions.Bson; + +import com.mongodb.ConnectionString; +import com.mongodb.client.ChangeStreamIterable; +import com.mongodb.client.MongoChangeStreamCursor; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.changestream.ChangeStreamDocument; +import com.mongodb.client.model.changestream.FullDocument; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import javax.annotation.Nonnull; + +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.regex.Pattern; + +import static com.mongodb.client.model.Aggregates.match; +import static com.mongodb.client.model.Filters.and; +import static com.mongodb.client.model.Filters.eq; +import static com.mongodb.client.model.Filters.or; +import static com.mongodb.client.model.Filters.regex; +import static com.mongodb.client.model.Projections.include; +import static com.mongodb.client.model.Sorts.ascending; +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ADD_NS_FIELD_NAME; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.COMMAND_SUCCEED_FLAG; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DROPPED_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MAX_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MIN_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SHARD_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UUID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.ADD_NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.includeListAsFlatPattern; + +@Slf4j +public class MongodbUtils { + + private static final Map> cache = new ConcurrentHashMap<>(); + + public static ChangeStreamDescriptor getChangeStreamDescriptor( + @Nonnull MongodbSourceConfig sourceConfig, + List discoveredDatabases, + List discoveredCollections) { + List databaseList = sourceConfig.getDatabaseList(); + List collectionList = sourceConfig.getCollectionList(); + + ChangeStreamDescriptor changeStreamFilter; + if (collectionList != null) { + // Watching collections changes + if (isIncludeListExplicitlySpecified(collectionList, discoveredCollections)) { + changeStreamFilter = + ChangeStreamDescriptor.collection( + TableId.parse(discoveredCollections.get(0))); + } else { + Pattern namespaceRegex = includeListAsFlatPattern(collectionList); + if (databaseList != null) { + if (isIncludeListExplicitlySpecified(databaseList, discoveredDatabases)) { + changeStreamFilter = + ChangeStreamDescriptor.database( + discoveredDatabases.get(0), namespaceRegex); + } else { + Pattern databaseRegex = includeListAsFlatPattern(databaseList); + changeStreamFilter = + ChangeStreamDescriptor.deployment(databaseRegex, namespaceRegex); + } + } else { + changeStreamFilter = ChangeStreamDescriptor.deployment(null, namespaceRegex); + } + } + } else if (databaseList != null) { + if (isIncludeListExplicitlySpecified(databaseList, discoveredDatabases)) { + changeStreamFilter = ChangeStreamDescriptor.database(discoveredDatabases.get(0)); + } else { + Pattern databaseRegex = includeListAsFlatPattern(databaseList); + changeStreamFilter = ChangeStreamDescriptor.deployment(databaseRegex); + } + } else { + // Watching all changes on the cluster + changeStreamFilter = ChangeStreamDescriptor.deployment(); + } + return changeStreamFilter; + } + + public static boolean isIncludeListExplicitlySpecified( + List includeList, List discoveredList) { + if (includeList == null || includeList.size() != 1) { + return false; + } + if (discoveredList == null || discoveredList.size() != 1) { + return false; + } + String firstOfIncludeList = includeList.get(0); + String firstOfDiscoveredList = discoveredList.get(0); + return firstOfDiscoveredList.equals(firstOfIncludeList); + } + + public static @Nonnull ChangeStreamIterable getChangeStreamIterable( + MongodbSourceConfig sourceConfig, @Nonnull ChangeStreamDescriptor descriptor) { + return getChangeStreamIterable( + createMongoClient(sourceConfig), + descriptor.getDatabase(), + descriptor.getCollection(), + descriptor.getDatabaseRegex(), + descriptor.getNamespaceRegex(), + sourceConfig.getBatchSize(), + sourceConfig.isUpdateLookup()); + } + + public static @Nonnull ChangeStreamIterable getChangeStreamIterable( + MongoClient mongoClient, + @Nonnull ChangeStreamDescriptor descriptor, + int batchSize, + boolean updateLookup) { + return getChangeStreamIterable( + mongoClient, + descriptor.getDatabase(), + descriptor.getCollection(), + descriptor.getDatabaseRegex(), + descriptor.getNamespaceRegex(), + batchSize, + updateLookup); + } + + public static @Nonnull ChangeStreamIterable getChangeStreamIterable( + MongoClient mongoClient, + String database, + String collection, + Pattern databaseRegex, + Pattern namespaceRegex, + int batchSize, + boolean updateLookup) { + ChangeStreamIterable changeStream; + if (StringUtils.isNotEmpty(database) && StringUtils.isNotEmpty(collection)) { + MongoCollection coll = + mongoClient.getDatabase(database).getCollection(collection); + log.info("Preparing change stream for collection {}.{}", database, collection); + changeStream = coll.watch(); + } else if (StringUtils.isNotEmpty(database) && namespaceRegex != null) { + MongoDatabase db = mongoClient.getDatabase(database); + List pipeline = new ArrayList<>(); + pipeline.add(ADD_NS_FIELD); + Bson nsFilter = regex(ADD_NS_FIELD_NAME, namespaceRegex); + pipeline.add(match(nsFilter)); + log.info( + "Preparing change stream for database {} with namespace regex filter {}", + database, + namespaceRegex); + changeStream = db.watch(pipeline); + } else if (StringUtils.isNotEmpty(database)) { + MongoDatabase db = mongoClient.getDatabase(database); + log.info("Preparing change stream for database {}", database); + changeStream = db.watch(); + } else if (namespaceRegex != null) { + List pipeline = new ArrayList<>(); + pipeline.add(ADD_NS_FIELD); + + Bson nsFilter = regex(ADD_NS_FIELD_NAME, namespaceRegex); + if (databaseRegex != null) { + Bson dbFilter = regex("ns.db", databaseRegex); + nsFilter = and(dbFilter, nsFilter); + log.info( + "Preparing change stream for deployment with" + + " database regex filter {} and namespace regex filter {}", + databaseRegex, + namespaceRegex); + } else { + log.info( + "Preparing change stream for deployment with namespace regex filter {}", + namespaceRegex); + } + + pipeline.add(match(nsFilter)); + changeStream = mongoClient.watch(pipeline); + } else if (databaseRegex != null) { + List pipeline = new ArrayList<>(); + pipeline.add(match(regex("ns.db", databaseRegex))); + + log.info( + "Preparing change stream for deployment with database regex filter {}", + databaseRegex); + changeStream = mongoClient.watch(pipeline); + } else { + log.info("Preparing change stream for deployment"); + changeStream = mongoClient.watch(); + } + + if (batchSize > 0) { + changeStream.batchSize(batchSize); + } + + if (updateLookup) { + changeStream.fullDocument(FullDocument.UPDATE_LOOKUP); + } + return changeStream; + } + + public static BsonDocument getLatestResumeToken( + MongoClient mongoClient, ChangeStreamDescriptor descriptor) { + ChangeStreamIterable changeStreamIterable = + getChangeStreamIterable(mongoClient, descriptor, 1, false); + + // Nullable when no change record or postResumeToken (new in MongoDB 4.0.7). + try (MongoChangeStreamCursor> changeStreamCursor = + changeStreamIterable.cursor()) { + ChangeStreamDocument firstResult = changeStreamCursor.tryNext(); + + return firstResult != null + ? firstResult.getResumeToken() + : changeStreamCursor.getResumeToken(); + } + } + + public static boolean isCommandSucceed(BsonDocument commandResult) { + return commandResult != null && COMMAND_SUCCEED_FLAG.equals(commandResult.getDouble("ok")); + } + + public static String commandErrorMessage(BsonDocument commandResult) { + return Optional.ofNullable(commandResult) + .map(doc -> doc.getString("errmsg")) + .map(BsonString::getValue) + .orElse(null); + } + + public static @Nonnull BsonDocument collStats( + @Nonnull MongoClient mongoClient, @Nonnull TableId collectionId) { + BsonDocument collStatsCommand = + new BsonDocument("collStats", new BsonString(collectionId.table())); + return mongoClient + .getDatabase(collectionId.catalog()) + .runCommand(collStatsCommand, BsonDocument.class); + } + + public static @Nonnull BsonDocument splitVector( + MongoClient mongoClient, + TableId collectionId, + BsonDocument keyPattern, + int maxChunkSizeMB) { + return splitVector(mongoClient, collectionId, keyPattern, maxChunkSizeMB, null, null); + } + + public static @Nonnull BsonDocument splitVector( + @Nonnull MongoClient mongoClient, + @Nonnull TableId collectionId, + BsonDocument keyPattern, + int maxChunkSizeMB, + BsonDocument min, + BsonDocument max) { + BsonDocument splitVectorCommand = + new BsonDocument("splitVector", new BsonString(collectionId.identifier())) + .append("keyPattern", keyPattern) + .append("maxChunkSize", new BsonInt32(maxChunkSizeMB)); + Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v)); + Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v)); + return mongoClient + .getDatabase(collectionId.catalog()) + .runCommand(splitVectorCommand, BsonDocument.class); + } + + public static BsonTimestamp getCurrentClusterTime(MongoClient mongoClient) { + BsonDocument isMasterResult = isMaster(mongoClient); + if (!isCommandSucceed(isMasterResult)) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Failed to execute isMaster command: " + commandErrorMessage(isMasterResult)); + } + return isMasterResult.getDocument("$clusterTime").getTimestamp("clusterTime"); + } + + public static @Nonnull BsonDocument isMaster(@Nonnull MongoClient mongoClient) { + BsonDocument isMasterCommand = new BsonDocument("isMaster", new BsonInt32(1)); + return mongoClient.getDatabase("admin").runCommand(isMasterCommand, BsonDocument.class); + } + + public static @Nonnull List readChunks( + MongoClient mongoClient, @Nonnull BsonDocument collectionMetadata) { + MongoCollection chunks = + getMongoCollection(mongoClient, TableId.parse("config.chunks"), BsonDocument.class); + List collectionChunks = new ArrayList<>(); + + Bson filter = + or( + new BsonDocument(NS_FIELD, collectionMetadata.get(ID_FIELD)), + // MongoDB 4.9.0 removed ns field of config.chunks collection, using + // collection's uuid instead. + // See: https://jira.mongodb.org/browse/SERVER-53105 + new BsonDocument(UUID_FIELD, collectionMetadata.get(UUID_FIELD))); + + chunks.find(filter) + .projection(include(MIN_FIELD, MAX_FIELD, SHARD_FIELD)) + .sort(ascending(MIN_FIELD)) + .into(collectionChunks); + return collectionChunks; + } + + public static BsonDocument readCollectionMetadata( + MongoClient mongoClient, @Nonnull TableId collectionId) { + MongoCollection collection = + getMongoCollection( + mongoClient, TableId.parse("config.collections"), BsonDocument.class); + + return collection + .find(eq(ID_FIELD, collectionId.identifier())) + .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, DOCUMENT_KEY)) + .first(); + } + + public static @Nonnull MongoCollection getMongoCollection( + MongoClient mongoClient, TableId collectionId, Class documentClass) { + return getCollection(mongoClient, collectionId, documentClass); + } + + @SuppressWarnings("unchecked") + public static @Nonnull MongoCollection getCollection( + MongoClient mongoClient, TableId collectionId, Class documentClass) { + MongoCollection cachedCollection = cache.get(collectionId); + if (cachedCollection == null) { + MongoCollection collection = + mongoClient + .getDatabase(collectionId.catalog()) + .getCollection(collectionId.table(), documentClass); + cache.put(collectionId, collection); + return collection; + } + return (MongoCollection) cachedCollection; + } + + public static MongoClient createMongoClient(MongodbSourceConfig sourceConfig) { + return MongodbClientProvider.INSTANCE.getOrCreateMongoClient(sourceConfig); + } + + public static @Nonnull ConnectionString buildConnectionString( + String username, String password, String hosts, String connectionOptions) { + StringBuilder sb = new StringBuilder("mongodb://"); + + if (hasCredentials(username, password)) { + appendCredentials(sb, username, password); + } + + sb.append(hosts); + + if (StringUtils.isNotEmpty(connectionOptions)) { + sb.append("/?").append(connectionOptions); + } + + return new ConnectionString(sb.toString()); + } + + private static boolean hasCredentials(String username, String password) { + return StringUtils.isNotEmpty(username) && StringUtils.isNotEmpty(password); + } + + private static void appendCredentials( + @Nonnull StringBuilder sb, String username, String password) { + sb.append(encodeValue(username)).append(":").append(encodeValue(password)).append("@"); + } + + public static String encodeValue(String value) { + try { + return URLEncoder.encode(value, StandardCharsets.UTF_8.name()); + } catch (UnsupportedEncodingException e) { + throw new MongodbConnectorException(ILLEGAL_ARGUMENT, e.getMessage()); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java new file mode 100644 index 000000000000..3ddd2ccbb21a --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import org.bson.BsonDocument; +import org.bson.BsonTimestamp; +import org.bson.BsonValue; + +import javax.annotation.Nonnull; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Objects; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; + +public class ResumeToken { + + private static final int K_TIMESTAMP = 130; + + public static @Nonnull BsonTimestamp decodeTimestamp(BsonDocument resumeToken) { + Objects.requireNonNull(resumeToken, "Missing ResumeToken."); + BsonValue bsonValue = resumeToken.get("_data"); + byte[] keyStringBytes = extractKeyStringBytes(bsonValue); + validateKeyType(keyStringBytes); + + ByteBuffer buffer = ByteBuffer.wrap(keyStringBytes).order(ByteOrder.BIG_ENDIAN); + int t = buffer.getInt(); + int i = buffer.getInt(); + return new BsonTimestamp(t, i); + } + + private static byte[] extractKeyStringBytes(@Nonnull BsonValue bsonValue) { + if (bsonValue.isBinary()) { + return bsonValue.asBinary().getData(); + } else if (bsonValue.isString()) { + return hexToUint8Array(bsonValue.asString().getValue()); + } else { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Unknown resume token format: " + bsonValue); + } + } + + private static void validateKeyType(byte[] keyStringBytes) { + int kType = keyStringBytes[0] & 0xff; + if (kType != K_TIMESTAMP) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Unknown keyType of timestamp: " + kType); + } + } + + private static byte[] hexToUint8Array(@Nonnull String str) { + int len = str.length(); + byte[] data = new byte[len / 2]; + for (int i = 0; i < len; i += 2) { + data[i / 2] = + (byte) + ((Character.digit(str.charAt(i), 16) << 4) + + Character.digit(str.charAt(i + 1), 16)); + } + return data; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/test/java/mongodb/source/MongodbIncrementalSourceFactoryTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/test/java/mongodb/source/MongodbIncrementalSourceFactoryTest.java new file mode 100644 index 000000000000..2269580f787a --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/test/java/mongodb/source/MongodbIncrementalSourceFactoryTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package mongodb.source; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.MongodbIncrementalSourceFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class MongodbIncrementalSourceFactoryTest { + @Test + public void testOptionRule() { + Assertions.assertNotNull((new MongodbIncrementalSourceFactory()).optionRule()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/testutils/UniqueDatabase.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/testutils/UniqueDatabase.java index e49ae3f9c566..c1a40bcfbb41 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/testutils/UniqueDatabase.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/testutils/UniqueDatabase.java @@ -50,7 +50,7 @@ public class UniqueDatabase { private static final String[] CREATE_DATABASE_DDL = - new String[] {"CREATE DATABASE $DBNAME$;", "USE $DBNAME$;"}; + new String[] {"CREATE DATABASE IF NOT EXISTS $DBNAME$;", "USE $DBNAME$;"}; private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); private final MySqlContainer container; @@ -82,6 +82,14 @@ private UniqueDatabase( this.password = password; } + public UniqueDatabase(MySqlContainer container, String databaseName) { + this.container = container; + this.databaseName = databaseName; + this.templateName = databaseName; + this.username = container.getUsername(); + this.password = container.getPassword(); + } + public String getHost() { return container.getHost(); } diff --git a/seatunnel-connectors-v2/connector-cdc/pom.xml b/seatunnel-connectors-v2/connector-cdc/pom.xml index 492dd7143626..38f52b1680d7 100644 --- a/seatunnel-connectors-v2/connector-cdc/pom.xml +++ b/seatunnel-connectors-v2/connector-cdc/pom.xml @@ -33,6 +33,7 @@ connector-cdc-base connector-cdc-mysql connector-cdc-sqlserver + connector-cdc-mongodb diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index fba4928996ad..fa1ac63c0bf5 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -464,6 +464,12 @@ ${project.version} provided + + org.apache.seatunnel + connector-cdc-mongodb + ${project.version} + provided + org.apache.seatunnel connector-cdc-sqlserver diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/pom.xml new file mode 100644 index 000000000000..a8814c11ee6c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/pom.xml @@ -0,0 +1,71 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-cdc-mongodb-e2e + SeaTunnel : E2E : Connector V2 : CDC Mongodb + + + 8.0.16 + + + + + org.apache.seatunnel + connector-cdc-mongodb + ${project.version} + test + + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test + + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test-jar + test + + + org.testcontainers + mysql + ${testcontainer.version} + test + + + org.apache.seatunnel + connector-jdbc + ${project.version} + test + + + mysql + mysql-connector-java + ${mysql.version} + test + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java new file mode 100644 index 000000000000..c33f6d047d48 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package mongodb; + +import org.apache.commons.lang3.StringUtils; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.containers.wait.strategy.WaitStrategy; +import org.testcontainers.images.builder.ImageFromDockerfile; + +import com.github.dockerjava.api.command.InspectContainerResponse; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertNotNull; + +@Slf4j +public class MongoDBContainer extends GenericContainer { + + private static final String DOCKER_IMAGE_NAME = "mongo:5.0.2"; + + public static final int MONGODB_PORT = 27017; + + public static final String MONGO_SUPER_USER = "superuser"; + + public static final String MONGO_SUPER_PASSWORD = "superpw"; + + private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)//.*$"); + + private final ShardingClusterRole clusterRole; + + public MongoDBContainer(Network network) { + this(network, ShardingClusterRole.NONE); + } + + public MongoDBContainer(Network network, ShardingClusterRole clusterRole) { + super( + new ImageFromDockerfile() + .withFileFromClasspath("random.key", "docker/mongodb/random.key") + .withFileFromClasspath("setup.js", "docker/mongodb/setup.js") + .withDockerfileFromBuilder( + builder -> + builder.from(DOCKER_IMAGE_NAME) + .copy( + "setup.js", + "/docker-entrypoint-initdb.d/setup.js") + .copy("random.key", "/data/keyfile/random.key") + .run("chown mongodb /data/keyfile/random.key") + .run("chmod 400 /data/keyfile/random.key") + .env("MONGO_INITDB_ROOT_USERNAME", MONGO_SUPER_USER) + .env( + "MONGO_INITDB_ROOT_PASSWORD", + MONGO_SUPER_PASSWORD) + .env("MONGO_INITDB_DATABASE", "admin") + .build())); + this.clusterRole = clusterRole; + + withNetwork(network); + withNetworkAliases(clusterRole.hostname); + withExposedPorts(MONGODB_PORT); + withCommand(ShardingClusterRole.startupCommand(clusterRole)); + waitingFor(clusterRole.waitStrategy); + } + + public void executeCommand(String command) { + try { + log.info("Executing mongo command: {}", command); + ExecResult execResult = + execInContainer( + "mongo", + "-u", + MONGO_SUPER_USER, + "-p", + MONGO_SUPER_PASSWORD, + "--eval", + command); + log.info(execResult.getStdout()); + if (execResult.getExitCode() != 0) { + throw new IllegalStateException( + "Execute mongo command failed " + execResult.getStdout()); + } + } catch (InterruptedException | IOException e) { + throw new IllegalStateException("Execute mongo command failed", e); + } + } + + @Override + protected void containerIsStarted(InspectContainerResponse containerInfo) { + log.info("Preparing a MongoDB Container with sharding cluster role {}...", clusterRole); + if (clusterRole != ShardingClusterRole.ROUTER) { + initReplicaSet(); + } else { + initShard(); + } + } + + protected void initReplicaSet() { + log.info("Initializing a single node replica set..."); + executeCommand( + String.format( + "rs.initiate({ _id : '%s', configsvr: %s, members: [{ _id: 0, host: '%s:%d'}]})", + clusterRole.replicaSetName, + clusterRole == ShardingClusterRole.CONFIG, + clusterRole.hostname, + MONGODB_PORT)); + + log.info("Waiting for single node replica set initialized..."); + executeCommand( + String.format( + "var attempt = 0; " + + "while" + + "(%s) " + + "{ " + + "if (attempt > %d) {quit(1);} " + + "print('%s ' + attempt); sleep(100); attempt++; " + + " }", + "db.runCommand( { isMaster: 1 } ).ismaster==false", + 60, + "An attempt to await for a single node replica set initialization:")); + } + + protected void initShard() { + log.info("Initializing a sharded cluster..."); + // decrease chunk size from default 64mb to 1mb to make splitter test easier. + executeCommand( + "db.getSiblingDB('config').settings.updateOne(\n" + + " { _id: \"chunksize\" },\n" + + " { $set: { _id: \"chunksize\", value: 1 } },\n" + + " { upsert: true }\n" + + ");"); + executeCommand( + String.format( + "sh.addShard('%s/%s:%d')", + ShardingClusterRole.SHARD.replicaSetName, + ShardingClusterRole.SHARD.hostname, + MONGODB_PORT)); + } + + public enum ShardingClusterRole { + // Config servers store metadata and configuration settings for the cluster. + CONFIG("config0", "rs0-config", Wait.forLogMessage(".*[Ww]aiting for connections.*", 2)), + + // Each shard contains a subset of the sharded data. Each shard can be deployed as a replica + // set. + SHARD("shard0", "rs0-shard", Wait.forLogMessage(".*[Ww]aiting for connections.*", 2)), + + // The mongos acts as a query router, providing an interface between client applications and + // the sharded cluster. + ROUTER("router0", null, Wait.forLogMessage(".*[Ww]aiting for connections.*", 1)), + + // None sharded cluster. + NONE("mongo0", "rs0", Wait.forLogMessage(".*Replication has not yet been configured.*", 1)); + + private final String hostname; + private final String replicaSetName; + private final WaitStrategy waitStrategy; + + ShardingClusterRole(String hostname, String replicaSetName, WaitStrategy waitStrategy) { + this.hostname = hostname; + this.replicaSetName = replicaSetName; + this.waitStrategy = waitStrategy; + } + + public static String startupCommand(ShardingClusterRole clusterRole) { + switch (clusterRole) { + case CONFIG: + return String.format( + "mongod --configsvr --port %d --replSet %s --keyFile /data/keyfile/random.key", + MONGODB_PORT, clusterRole.replicaSetName); + case SHARD: + return String.format( + "mongod --shardsvr --port %d --replSet %s --keyFile /data/keyfile/random.key", + MONGODB_PORT, clusterRole.replicaSetName); + case ROUTER: + return String.format( + "mongos --configdb %s/%s:%d --bind_ip_all --keyFile /data/keyfile/random.key", + CONFIG.replicaSetName, CONFIG.hostname, MONGODB_PORT); + case NONE: + default: + return String.format( + "mongod --port %d --replSet %s --keyFile /data/keyfile/random.key", + MONGODB_PORT, NONE.replicaSetName); + } + } + } + + public void executeCommandFileInSeparateDatabase(String fileNameIgnoreSuffix) { + executeCommandFileInDatabase(fileNameIgnoreSuffix, fileNameIgnoreSuffix); + } + + public void executeCommandFileInDatabase(String fileNameIgnoreSuffix, String databaseName) { + final String dbName = databaseName != null ? databaseName : fileNameIgnoreSuffix; + final String ddlFile = String.format("ddl/%s.js", fileNameIgnoreSuffix); + final URL ddlTestFile = MongoDBContainer.class.getClassLoader().getResource(ddlFile); + assertNotNull("Cannot locate " + ddlFile, ddlTestFile); + + try { + // use database; + String command0 = String.format("db = db.getSiblingDB('%s');\n", dbName); + String command1 = + Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() + .filter(x -> StringUtils.isNotBlank(x) && !x.trim().startsWith("//")) + .map( + x -> { + final Matcher m = COMMENT_PATTERN.matcher(x); + return m.matches() ? m.group(1) : x; + }) + .collect(Collectors.joining("\n")); + + executeCommand(command0 + command1); + + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java new file mode 100644 index 000000000000..dd7f985f1760 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java @@ -0,0 +1,266 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package mongodb; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlContainer; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlVersion; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.UniqueDatabase; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.bson.Document; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.lifecycle.Startables; + +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.model.Sorts; +import lombok.extern.slf4j.Slf4j; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.await; + +@Slf4j +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "Currently SPARK and FLINK do not support cdc") +public class MongodbCDCIT extends TestSuiteBase implements TestResource { + + // ---------------------------------------------------------------------------- + // mongodb + protected static final String MONGODB_DATABASE = "inventory"; + + protected static final String MONGODB_COLLECTION = "products"; + protected MongoDBContainer mongodbContainer; + + protected MongoClient client; + + // ---------------------------------------------------------------------------- + // mysql + private static final String MYSQL_HOST = "mysql_cdc_e2e"; + + private static final String MYSQL_USER_NAME = "st_user"; + + private static final String MYSQL_USER_PASSWORD = "seatunnel"; + + private static final String MYSQL_DATABASE = "mongodb_cdc"; + + private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(); + + // mysql sink table query sql + private static final String SINK_SQL = "select name,description,weight from products"; + + private static final String MYSQL_DRIVER_JAR = + "https://repo1.maven.org/maven2/mysql/mysql-connector-java/8.0.16/mysql-connector-java-8.0.16.jar"; + + private final UniqueDatabase inventoryDatabase = + new UniqueDatabase(MYSQL_CONTAINER, MYSQL_DATABASE); + + private static MySqlContainer createMySqlContainer() { + MySqlContainer mySqlContainer = new MySqlContainer(MySqlVersion.V8_0); + mySqlContainer.withNetwork(NETWORK); + mySqlContainer.withNetworkAliases(MYSQL_HOST); + mySqlContainer.withDatabaseName(MYSQL_DATABASE); + mySqlContainer.withUsername(MYSQL_USER_NAME); + mySqlContainer.withPassword(MYSQL_USER_PASSWORD); + // For local test use + // mySqlContainer.setPortBindings(Collections.singletonList("3308:3306")); + return mySqlContainer; + } + + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && wget " + + MYSQL_DRIVER_JAR); + Assertions.assertEquals(0, extraCommands.getExitCode(), extraCommands.getStderr()); + }; + + @BeforeAll + @Override + public void startUp() { + log.info("The first stage:Starting Mysql containers..."); + Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); + log.info("Mysql Containers are started"); + inventoryDatabase.createAndInitialize(); + log.info("Mysql ddl-a execution is complete"); + + log.info("The second stage:Starting Mongodb containers..."); + mongodbContainer = new MongoDBContainer(NETWORK); + // For local test use + mongodbContainer.setPortBindings(Collections.singletonList("27017:27017")); + Startables.deepStart(Stream.of(mongodbContainer)).join(); + mongodbContainer.executeCommandFileInSeparateDatabase(MONGODB_DATABASE); + initConnection(); + log.info("Mongodb Container are started"); + } + + @TestTemplate + public void testMongodbCdcToMysqlCheckDataE2e(TestContainer container) { + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob("/mongodbcdc_to_mysql.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(); + } + return null; + }); + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + readMongodbData().stream() + .peek(e -> e.remove("_id")) + .map(Document::entrySet) + .map(Set::stream) + .map( + entryStream -> + entryStream + .map(Map.Entry::getValue) + .collect( + Collectors.toCollection( + ArrayList + ::new))) + .collect(Collectors.toList()), + querySql()); + }); + + // insert update delete + upsertDeleteSourceTable(); + + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + readMongodbData().stream() + .peek(e -> e.remove("_id")) + .map(Document::entrySet) + .map(Set::stream) + .map( + entryStream -> + entryStream + .map(Map.Entry::getValue) + .collect( + Collectors.toCollection( + ArrayList + ::new))) + .collect(Collectors.toList()), + querySql()); + }); + } + + private Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection( + MYSQL_CONTAINER.getJdbcUrl(), + MYSQL_CONTAINER.getUsername(), + MYSQL_CONTAINER.getPassword()); + } + + private List> querySql() { + try (Connection connection = getJdbcConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(MongodbCDCIT.SINK_SQL); + List> result = new ArrayList<>(); + int columnCount = resultSet.getMetaData().getColumnCount(); + while (resultSet.next()) { + ArrayList objects = new ArrayList<>(); + for (int i = 1; i <= columnCount; i++) { + objects.add(resultSet.getObject(i)); + } + result.add(objects); + } + return result; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private void upsertDeleteSourceTable() { + mongodbContainer.executeCommandFileInDatabase("inventoryDDL", MONGODB_DATABASE); + } + + public void initConnection() { + String ipAddress = mongodbContainer.getHost(); + Integer port = mongodbContainer.getFirstMappedPort(); + String url = + String.format( + "mongodb://%s:%s@%s:%d/%s?authSource=admin", + "superuser", + "superpw", + ipAddress, + port, + MONGODB_DATABASE + "." + MONGODB_COLLECTION); + client = MongoClients.create(url); + } + + protected List readMongodbData() { + MongoCollection sinkTable = + client.getDatabase(MONGODB_DATABASE).getCollection(MongodbCDCIT.MONGODB_COLLECTION); + // If the cursor has been traversed, it will automatically close without explicitly closing. + MongoCursor cursor = sinkTable.find().sort(Sorts.ascending("_id")).cursor(); + List documents = new ArrayList<>(); + while (cursor.hasNext()) { + documents.add(cursor.next()); + } + return documents; + } + + @AfterAll + @Override + public void tearDown() { + // close Container + if (Objects.nonNull(client)) { + client.close(); + } + MYSQL_CONTAINER.close(); + if (mongodbContainer != null) { + mongodbContainer.close(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventory.js b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventory.js new file mode 100644 index 000000000000..c834ec8a2cec --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventory.js @@ -0,0 +1,24 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// -- this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000101"), "name": "scooter", "description": "Small 2-wheel scooter", "weight": "314"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000102"), "name": "car battery", "description": "12V car battery", "weight": "81"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000103"), "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": "8"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000104"), "name": "hammer", "description": "12oz carpenter''s hammer", "weight": "75"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000105"), "name": "hammer", "description": "12oz carpenter''s hammer", "weight": "875"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000106"), "name": "hammer", "description": "12oz carpenter''s hammer", "weight": "10"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000107"), "name": "rocks", "description": "box of assorted rocks", "weight": "53"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000108"), "name": "jacket", "description": "water resistent black wind breaker", "weight": "1"}); + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventoryDDL.js b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventoryDDL.js new file mode 100644 index 000000000000..db05f5f59fff --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventoryDDL.js @@ -0,0 +1,32 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// -- this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000109"), "name": "bicycle", "description": "Mountain bike with 21 gears", "weight": "1200"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000110"), "name": "headphones", "description": "Wireless headphones with noise cancellation", "weight": "200"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000111"), "name": "laptop", "description": "13-inch ultrabook with 16GB RAM and SSD storage", "weight": "1100"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000112"), "name": "blender", "description": "High-powered blender for smoothies and shakes", "weight": "400"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000113"), "name": "notebook", "description": "Spiral-bound notebook with ruled pages", "weight": "300"}); + +db.getCollection('products').updateOne({"name": "scooter"}, {$set: {"weight": "350"}}); +db.getCollection('products').updateOne({"name": "car battery"}, {$set: {"description": "High-performance car battery"}}); +db.getCollection('products').updateOne({"name": "12-pack drill bits"}, {$set: {"description": "Set of 12 professional-grade drill bits"}}); +db.getCollection('products').updateOne({"name": "hammer"}, {$set: {"weight": "100"}}); +db.getCollection('products').updateOne({"name": "rocks"}, {$set: {"weight": "1000"}}); + +db.getCollection('products').deleteOne({"_id": ObjectId("100000000000000000000101")}); +db.getCollection('products').deleteOne({"name": "car battery"}); +db.getCollection('products').deleteOne({"name": "12-pack drill bits"}); +db.getCollection('products').deleteOne({"name": "hammer", "weight": "875"}); +db.getCollection('products').deleteOne({"name": "jacket"}); \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/mongodb_cdc.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/mongodb_cdc.sql new file mode 100644 index 000000000000..cc7a619af69d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/mongodb_cdc.sql @@ -0,0 +1,32 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: mongodb_cdc +-- ---------------------------------------------------------------------------------------------------------------- +CREATE DATABASE IF NOT EXISTS `mongodb_cdc`; + +use mongodb_cdc; + +-- Create and populate our products using a single insert with many rows +CREATE TABLE products ( + _id VARCHAR(512) NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL, + description VARCHAR(512), + weight VARCHAR(255) +); + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/random.key b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/random.key new file mode 100644 index 000000000000..7398791e39d1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/random.key @@ -0,0 +1,34 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +XK8G9pNKhEPp/BlsKT7pHEc5i0oCpvNVZMALH5pD/6EHSuMzuyO1FpoeDwmWHXl0 ++Gp+VOI89Xp7E6eqop+fFHtoM3Mnk2oTiI/442GvS0xISPTwFVY9nO3MfO8VcPVx +J3JCAb80GeXD5x55eAOi7NqXzpjk0OKqfPEwIn1lrjlkL2m5vq6kaKEd93i1+bMh +3LRd1jLbgwWWxqYVV92BTQNnJin+G1er7Y2FzLpeFIKqyy+I22qIE2XIC7yj3wSw +kxwKsPN5LjFsfVeKpf169R0KgBg4Nm0qlllVUGNKuEjaVoLOEBOJgoPnhC6L2avc +/iDeunZDlDDgYG6t6aJXJelP+W1uXp4JQj1j18Scn0lrvgWxdAVrAtK6ftxqutHc +RQBt6Ap63zojTraulm3aeo/w/yz0zjyYjxQ5t8cojIM/7TaNLe2GfVxwhqitUPL1 +ct2YFXWwX1H/+8E7yTsnquKqe6+r0aGQqxS5x+wFMsDun/1mxv7jgjwzZc1rEk8H +DGdhnQ7MFPOE6Bp03zGpa6B6K4I5uDgUUeOC7zmAN63cPEumuuCjPVK42sMt5wwR +NPJyL4+sWHa9vb2sBJ1dk3thQ+wwz856BZ9ILgeMUutQgasSwctlI7t3rhM+BGYy ++naEhKWN9/cIDXtl3ZMhNWJIh/MqbluYazQ/97MZHeWc9CJXFU6yUrnJOdE0VvQd +tROQNDuEB0Tq9ITxSYpZTY49+1CQp5E14GIc8frieWPvcbNVknriFquQfsW/tMvk +V2Aj8sBYE+sW9sGQJlyfRrhTSN6aBG1em7ZkOAgcx2/5ftaEZTwBxNnJR9VZDYEi +CDbobs3hIX3qhS6J9YbTEPFF2L6MMTL3ADgS44cWtmlYQrb2HJT0YLmdCzk4lSa6 +yWYLorduRtblgGo6v/nn7y41gn/l/aRdcDUsii/LgMco4ZPSRm0HixD8oA3agX9/ +23M5UVNCBO4/RKFOnjWM/2tN1xjeQrS2Hn6j3BtoTOl6k4ho diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/setup.js b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/setup.js new file mode 100644 index 000000000000..9cef6b876b95 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/setup.js @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// -- this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//use admin; +db.createRole( + { + role: "strole", + privileges: [{ + // Grant privileges on All Non-System Collections in All Databases + resource: {db: "", collection: ""}, + actions: ["splitVector", "listDatabases", "listCollections", "collStats", "find", "changeStream"] + }], + roles: [ + {role: 'read', db: 'config'} + ] + } +); + +db.createUser( + { + user: 'stuser', + pwd: 'stpw', + roles: [ + {role: 'strole', db: 'admin'} + ] + } +); \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/log4j2-test.properties new file mode 100644 index 000000000000..d1ca535f2be6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/log4j2-test.properties @@ -0,0 +1,29 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level=INFO +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c - %m%n diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf new file mode 100644 index 000000000000..7e4a492390ba --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + execution.checkpoint.interval = 5000 +} + +source { + MongoDB-CDC { + hosts = "mongo0:27017" + database = ["inventory"] + collection = ["inventory.products"] + username = superuser + password = superpw + schema = { + fields { + "_id": string, + "name": string, + "description": string, + "weight": string + } + } + } +} + +sink { + jdbc { + url = "jdbc:mysql://mysql_cdc_e2e:3306?useSSL=false&useUnicode=true&characterEncoding=UTF-8&allowPublicKeyRetrieval=false&useJDBCCompliantTimezoneShift=true&useLegacyDatetimeCode=false&serverTimezone=GMT%2B8" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user" + password = "seatunnel" + + generate_sink_sql = true + # You need to configure both database and table + database = mongodb_cdc + table = products + primary_keys = ["_id"] + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index a173e35e9284..65798fee100c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -47,6 +47,7 @@ connector-elasticsearch-e2e connector-iotdb-e2e connector-cdc-mysql-e2e + connector-cdc-mongodb-e2e connector-iceberg-e2e connector-iceberg-hadoop3-e2e connector-tdengine-e2e From 4acd370d488f8a7101591911a65e47b5ddb76249 Mon Sep 17 00:00:00 2001 From: Eric Date: Mon, 17 Jul 2023 19:05:48 +0800 Subject: [PATCH 02/43] [Improve] Improve savemode api (#4767) --- .../seatunnel/api/configuration/Options.java | 4 +- .../api/configuration/SingleChoiceOption.java | 8 ++- .../seatunnel/api/sink/DataSaveMode.java | 4 ++ .../seatunnel/api/sink/SinkCommonOptions.java | 23 --------- .../api/sink/SupportDataSaveMode.java | 51 +++---------------- .../api/table/factory/FactoryUtil.java | 28 ---------- .../seatunnel/jdbc/sink/JdbcSink.java | 8 +-- .../starrocks/config/SinkConfig.java | 4 ++ .../config/StarRocksSinkOptions.java | 12 +++++ .../starrocks/sink/StarRocksSink.java | 17 ++----- .../starrocks/sink/StarRocksSinkFactory.java | 4 +- .../flink/execution/SinkExecuteProcessor.java | 8 +-- .../flink/execution/SinkExecuteProcessor.java | 8 +-- .../spark/execution/SinkExecuteProcessor.java | 8 +-- .../spark/execution/SinkExecuteProcessor.java | 8 +-- .../parse/MultipleTableJobConfigParser.java | 2 +- 16 files changed, 47 insertions(+), 150 deletions(-) delete mode 100644 seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SinkCommonOptions.java diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/Options.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/Options.java index 432e931c235f..a4ce408d73b0 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/Options.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/Options.java @@ -249,7 +249,7 @@ public static class SingleChoiceOptionBuilder { * @param value The default value for the config option * @return The config option with the default value. */ - public Option defaultValue(T value) { + public SingleChoiceOption defaultValue(T value) { return new SingleChoiceOption(key, typeReference, optionValues, value); } @@ -258,7 +258,7 @@ public Option defaultValue(T value) { * * @return The config option without a default value. */ - public Option noDefaultValue() { + public SingleChoiceOption noDefaultValue() { return new SingleChoiceOption(key, typeReference, optionValues, null); } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/SingleChoiceOption.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/SingleChoiceOption.java index fd3697f681f4..b3a6574e9ed7 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/SingleChoiceOption.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/configuration/SingleChoiceOption.java @@ -23,7 +23,7 @@ import java.util.List; -public class SingleChoiceOption extends Option { +public class SingleChoiceOption extends Option { @Getter private final List optionValues; @@ -32,4 +32,10 @@ public SingleChoiceOption( super(key, typeReference, defaultValue); this.optionValues = optionValues; } + + @Override + public SingleChoiceOption withDescription(String description) { + this.description = description; + return this; + } } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DataSaveMode.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DataSaveMode.java index f269c9f2cb78..7ef849f6147f 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DataSaveMode.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/DataSaveMode.java @@ -31,6 +31,10 @@ public enum DataSaveMode { // path and files in the path, create new files in the path. KEEP_SCHEMA_AND_DATA, + // The connector provides custom processing methods, such as running user provided SQL or shell + // scripts, etc + CUSTOM_PROCESSING, + // Throw error when table is exists for MySQL. Throw error when path is exists. ERROR_WHEN_EXISTS } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SinkCommonOptions.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SinkCommonOptions.java deleted file mode 100644 index 4bf320b49cbd..000000000000 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SinkCommonOptions.java +++ /dev/null @@ -1,23 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.api.sink; - -public class SinkCommonOptions { - - public static final String DATA_SAVE_MODE = "save_mode"; -} diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SupportDataSaveMode.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SupportDataSaveMode.java index 7d0c2838befb..46ea2e70e53b 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SupportDataSaveMode.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/sink/SupportDataSaveMode.java @@ -17,55 +17,16 @@ package org.apache.seatunnel.api.sink; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; -import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; - -import java.util.List; -import java.util.Locale; - /** The Sink Connectors which support data SaveMode should implement this interface */ public interface SupportDataSaveMode { - - /** - * We hope every sink connector use the same option name to config SaveMode, So I add - * checkOptions method to this interface. checkOptions method have a default implement to check - * whether `save_mode` parameter is in config. - * - * @param config config of sink Connector - */ - default void checkOptions(Config config) { - if (config.hasPath(SinkCommonOptions.DATA_SAVE_MODE)) { - String tableSaveMode = config.getString(SinkCommonOptions.DATA_SAVE_MODE); - DataSaveMode dataSaveMode = - DataSaveMode.valueOf(tableSaveMode.toUpperCase(Locale.ROOT)); - if (!supportedDataSaveModeValues().contains(dataSaveMode)) { - throw new SeaTunnelRuntimeException( - SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - "This connector don't support save mode: " + dataSaveMode); - } - } - } - + String SAVE_MODE_KEY = "savemode"; /** - * Get the {@link DataSaveMode} that the user configured + * Return the value of DataSaveMode configured by user in the job config file. * - * @return DataSaveMode + * @return */ - DataSaveMode getDataSaveMode(); + DataSaveMode getUserConfigSaveMode(); - /** - * Return the {@link DataSaveMode} list supported by this connector - * - * @return the list of supported data save modes - */ - List supportedDataSaveModeValues(); - - /** - * The implementation of specific logic according to different {@link DataSaveMode} - * - * @param saveMode data save mode - */ - void handleSaveMode(DataSaveMode saveMode); + /** The implementation of specific logic according to different {@link DataSaveMode} */ + void handleSaveMode(DataSaveMode userConfigSaveMode); } diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java index 6ac939149c71..f30900269912 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/factory/FactoryUtil.java @@ -18,21 +18,15 @@ package org.apache.seatunnel.api.table.factory; import org.apache.seatunnel.api.common.CommonOptions; -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.ConfigValidator; import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.sink.DataSaveMode; import org.apache.seatunnel.api.sink.SeaTunnelSink; -import org.apache.seatunnel.api.sink.SinkCommonOptions; -import org.apache.seatunnel.api.sink.SupportDataSaveMode; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.source.SupportParallelism; import org.apache.seatunnel.api.table.catalog.Catalog; import org.apache.seatunnel.api.table.catalog.CatalogTable; -import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.connector.TableSource; import org.apache.seatunnel.api.transform.SeaTunnelTransform; @@ -289,28 +283,6 @@ public static OptionRule sinkFullOptionRule(@NonNull TableSinkFactory factory) { if (sinkOptionRule == null) { throw new FactoryException("sinkOptionRule can not be null"); } - - try { - TableSink sink = factory.createSink(null); - if (SupportDataSaveMode.class.isAssignableFrom(sink.getClass())) { - SupportDataSaveMode supportDataSaveModeSink = (SupportDataSaveMode) sink; - Option saveMode = - Options.key(SinkCommonOptions.DATA_SAVE_MODE) - .singleChoice( - DataSaveMode.class, - supportDataSaveModeSink.supportedDataSaveModeValues()) - .noDefaultValue() - .withDescription("data save mode"); - OptionRule sinkCommonOptionRule = OptionRule.builder().required(saveMode).build(); - sinkOptionRule - .getOptionalOptions() - .addAll(sinkCommonOptionRule.getOptionalOptions()); - } - } catch (Exception e) { - LOG.warn( - "Add save mode option need sink connector support create sink by TableSinkFactory"); - } - return sinkOptionRule; } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java index 4666eae1e51a..eec473512b77 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java @@ -52,7 +52,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -182,15 +181,10 @@ public Optional> getCommitInfoSerializer() { } @Override - public DataSaveMode getDataSaveMode() { + public DataSaveMode getUserConfigSaveMode() { return dataSaveMode; } - @Override - public List supportedDataSaveModeValues() { - return Collections.singletonList(DataSaveMode.KEEP_SCHEMA_AND_DATA); - } - @Override public void handleSaveMode(DataSaveMode saveMode) { if (catalogTable != null) { diff --git a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/config/SinkConfig.java b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/config/SinkConfig.java index 91bfb9358b17..f5a2d0dc88c1 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/config/SinkConfig.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/config/SinkConfig.java @@ -18,6 +18,7 @@ package org.apache.seatunnel.connectors.seatunnel.starrocks.config; import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.sink.DataSaveMode; import lombok.Getter; import lombok.Setter; @@ -58,6 +59,8 @@ public enum StreamLoadFormat { private String saveModeCreateTemplate; + private DataSaveMode dataSaveMode; + @Getter private final Map streamLoadProps = new HashMap<>(); public static SinkConfig of(ReadonlyConfig config) { @@ -89,6 +92,7 @@ public static SinkConfig of(ReadonlyConfig config) { config.getOptional(StarRocksSinkOptions.COLUMN_SEPARATOR) .ifPresent(sinkConfig::setColumnSeparator); sinkConfig.setLoadFormat(config.get(StarRocksSinkOptions.LOAD_FORMAT)); + sinkConfig.setDataSaveMode(config.get(StarRocksSinkOptions.SAVE_MODE)); return sinkConfig; } } diff --git a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/config/StarRocksSinkOptions.java b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/config/StarRocksSinkOptions.java index 4f87b690f187..02918f0f96d7 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/config/StarRocksSinkOptions.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/config/StarRocksSinkOptions.java @@ -19,8 +19,12 @@ import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.configuration.SingleChoiceOption; +import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.sink.SupportDataSaveMode; import org.apache.seatunnel.connectors.seatunnel.starrocks.config.SinkConfig.StreamLoadFormat; +import java.util.Arrays; import java.util.List; import java.util.Map; @@ -133,4 +137,12 @@ public interface StarRocksSinkOptions { .enumType(StreamLoadFormat.class) .defaultValue(StreamLoadFormat.JSON) .withDescription(""); + + SingleChoiceOption SAVE_MODE = + Options.key(SupportDataSaveMode.SAVE_MODE_KEY) + .singleChoice( + DataSaveMode.class, Arrays.asList(DataSaveMode.KEEP_SCHEMA_AND_DATA)) + .defaultValue(DataSaveMode.KEEP_SCHEMA_AND_DATA) + .withDescription( + "Table structure and data processing methods that already exist on the target end"); } diff --git a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/sink/StarRocksSink.java b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/sink/StarRocksSink.java index ae808a36eef7..54163bd6f1de 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/sink/StarRocksSink.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/sink/StarRocksSink.java @@ -42,9 +42,6 @@ import com.google.auto.service.AutoService; import lombok.NoArgsConstructor; -import java.util.Collections; -import java.util.List; - @NoArgsConstructor @AutoService(SeaTunnelSink.class) public class StarRocksSink extends AbstractSimpleSink @@ -56,12 +53,11 @@ public class StarRocksSink extends AbstractSimpleSink private CatalogTable catalogTable; - public StarRocksSink( - DataSaveMode dataSaveMode, SinkConfig sinkConfig, CatalogTable catalogTable) { - this.dataSaveMode = dataSaveMode; + public StarRocksSink(SinkConfig sinkConfig, CatalogTable catalogTable) { this.sinkConfig = sinkConfig; this.seaTunnelRowType = catalogTable.getTableSchema().toPhysicalRowDataType(); this.catalogTable = catalogTable; + this.dataSaveMode = sinkConfig.getDataSaveMode(); } @Override @@ -77,7 +73,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { if (StringUtils.isEmpty(sinkConfig.getTable()) && catalogTable != null) { sinkConfig.setTable(catalogTable.getTableId().getTableName()); } - dataSaveMode = DataSaveMode.KEEP_SCHEMA_AND_DATA; + dataSaveMode = sinkConfig.getDataSaveMode(); } private void autoCreateTable(String template) { @@ -117,15 +113,10 @@ public AbstractSinkWriter createWriter(SinkWriter.Context co } @Override - public DataSaveMode getDataSaveMode() { + public DataSaveMode getUserConfigSaveMode() { return dataSaveMode; } - @Override - public List supportedDataSaveModeValues() { - return Collections.singletonList(DataSaveMode.KEEP_SCHEMA_AND_DATA); - } - @Override public void handleSaveMode(DataSaveMode saveMode) { if (catalogTable != null) { diff --git a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/sink/StarRocksSinkFactory.java b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/sink/StarRocksSinkFactory.java index 44a84c54898b..471be7001b68 100644 --- a/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/sink/StarRocksSinkFactory.java +++ b/seatunnel-connectors-v2/connector-starrocks/src/main/java/org/apache/seatunnel/connectors/seatunnel/starrocks/sink/StarRocksSinkFactory.java @@ -18,7 +18,6 @@ package org.apache.seatunnel.connectors.seatunnel.starrocks.sink; import org.apache.seatunnel.api.configuration.util.OptionRule; -import org.apache.seatunnel.api.sink.DataSaveMode; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -56,6 +55,7 @@ public OptionRule optionRule() { StarRocksSinkOptions.RETRY_BACKOFF_MULTIPLIER_MS, StarRocksSinkOptions.STARROCKS_CONFIG, StarRocksSinkOptions.ENABLE_UPSERT_DELETE, + StarRocksSinkOptions.SAVE_MODE, StarRocksSinkOptions.SAVE_MODE_CREATE_TEMPLATE) .build(); } @@ -67,6 +67,6 @@ public TableSink createSink(TableFactoryContext context) { if (StringUtils.isBlank(sinkConfig.getTable())) { sinkConfig.setTable(catalogTable.getTableId().getTableName()); } - return () -> new StarRocksSink(DataSaveMode.KEEP_SCHEMA_AND_DATA, sinkConfig, catalogTable); + return () -> new StarRocksSink(sinkConfig, catalogTable); } } diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java index c23225538652..03bd2077e500 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-13-starter/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java @@ -84,12 +84,6 @@ protected SinkExecuteProcessor( pluginIdentifier); seaTunnelSink.prepare(sinkConfig); seaTunnelSink.setJobContext(jobContext); - if (SupportDataSaveMode.class.isAssignableFrom( - seaTunnelSink.getClass())) { - SupportDataSaveMode saveModeSink = - (SupportDataSaveMode) seaTunnelSink; - saveModeSink.checkOptions(sinkConfig); - } return seaTunnelSink; }) .distinct() @@ -111,7 +105,7 @@ public List> execute(List> upstreamDataStreams) (SeaTunnelRowType) TypeConverterUtils.convert(stream.getType())); if (SupportDataSaveMode.class.isAssignableFrom(seaTunnelSink.getClass())) { SupportDataSaveMode saveModeSink = (SupportDataSaveMode) seaTunnelSink; - DataSaveMode dataSaveMode = saveModeSink.getDataSaveMode(); + DataSaveMode dataSaveMode = saveModeSink.getUserConfigSaveMode(); saveModeSink.handleSaveMode(dataSaveMode); } DataStreamSink dataStreamSink = diff --git a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java index d8fa8eeddff3..ca9a05f632a6 100644 --- a/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-flink-starter/seatunnel-flink-starter-common/src/main/java/org/apache/seatunnel/core/starter/flink/execution/SinkExecuteProcessor.java @@ -85,12 +85,6 @@ protected SinkExecuteProcessor( pluginIdentifier); seaTunnelSink.prepare(sinkConfig); seaTunnelSink.setJobContext(jobContext); - if (SupportDataSaveMode.class.isAssignableFrom( - seaTunnelSink.getClass())) { - SupportDataSaveMode saveModeSink = - (SupportDataSaveMode) seaTunnelSink; - saveModeSink.checkOptions(sinkConfig); - } return seaTunnelSink; }) .distinct() @@ -112,7 +106,7 @@ public List> execute(List> upstreamDataStreams) (SeaTunnelRowType) TypeConverterUtils.convert(stream.getType())); if (SupportDataSaveMode.class.isAssignableFrom(seaTunnelSink.getClass())) { SupportDataSaveMode saveModeSink = (SupportDataSaveMode) seaTunnelSink; - DataSaveMode dataSaveMode = saveModeSink.getDataSaveMode(); + DataSaveMode dataSaveMode = saveModeSink.getUserConfigSaveMode(); saveModeSink.handleSaveMode(dataSaveMode); } DataStreamSink dataStreamSink = diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java index 61f3fb07b6a4..503f76b87a16 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-2-starter/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java @@ -75,12 +75,6 @@ protected SinkExecuteProcessor( pluginIdentifier); seaTunnelSink.prepare(sinkConfig); seaTunnelSink.setJobContext(jobContext); - if (SupportDataSaveMode.class.isAssignableFrom( - seaTunnelSink.getClass())) { - SupportDataSaveMode saveModeSink = - (SupportDataSaveMode) seaTunnelSink; - saveModeSink.checkOptions(sinkConfig); - } return seaTunnelSink; }) .distinct() @@ -115,7 +109,7 @@ public List> execute(List> upstreamDataStreams) (SeaTunnelRowType) TypeConverterUtils.convert(dataset.schema())); if (SupportDataSaveMode.class.isAssignableFrom(seaTunnelSink.getClass())) { SupportDataSaveMode saveModeSink = (SupportDataSaveMode) seaTunnelSink; - DataSaveMode dataSaveMode = saveModeSink.getDataSaveMode(); + DataSaveMode dataSaveMode = saveModeSink.getUserConfigSaveMode(); saveModeSink.handleSaveMode(dataSaveMode); } SparkSinkInjector.inject(dataset.write(), seaTunnelSink) diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java index f4d3c0b15b57..8afffe1add0b 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/SinkExecuteProcessor.java @@ -76,12 +76,6 @@ protected SinkExecuteProcessor( pluginIdentifier); seaTunnelSink.prepare(sinkConfig); seaTunnelSink.setJobContext(jobContext); - if (SupportDataSaveMode.class.isAssignableFrom( - seaTunnelSink.getClass())) { - SupportDataSaveMode saveModeSink = - (SupportDataSaveMode) seaTunnelSink; - saveModeSink.checkOptions(sinkConfig); - } return seaTunnelSink; }) .distinct() @@ -116,7 +110,7 @@ public List> execute(List> upstreamDataStreams) (SeaTunnelRowType) TypeConverterUtils.convert(dataset.schema())); if (SupportDataSaveMode.class.isAssignableFrom(seaTunnelSink.getClass())) { SupportDataSaveMode saveModeSink = (SupportDataSaveMode) seaTunnelSink; - DataSaveMode dataSaveMode = saveModeSink.getDataSaveMode(); + DataSaveMode dataSaveMode = saveModeSink.getUserConfigSaveMode(); saveModeSink.handleSaveMode(dataSaveMode); } SparkSinkInjector.inject(dataset.write(), seaTunnelSink) diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java index 5fd4892cd09e..faf178e1bdd1 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java @@ -616,7 +616,7 @@ private static T findLast(LinkedHashMap map) { public static void handleSaveMode(SeaTunnelSink sink) { if (SupportDataSaveMode.class.isAssignableFrom(sink.getClass())) { SupportDataSaveMode saveModeSink = (SupportDataSaveMode) sink; - DataSaveMode dataSaveMode = saveModeSink.getDataSaveMode(); + DataSaveMode dataSaveMode = saveModeSink.getUserConfigSaveMode(); saveModeSink.handleSaveMode(dataSaveMode); } } From 3aa4ae5e7c5ddfcb54afe80743f71953678840a0 Mon Sep 17 00:00:00 2001 From: zhangchengming601 <86779821+zhangchengming601@users.noreply.github.com> Date: Tue, 18 Jul 2023 21:34:21 +0800 Subject: [PATCH 03/43] [Doc] Improve DB2 Source Vertica Source & DB2 Sink Vertica Sink document (#5102) --- docs/en/connector-v2/sink/DB2.md | 171 ++++++++++++++++++++++++ docs/en/connector-v2/sink/Vertica.md | 173 +++++++++++++++++++++++++ docs/en/connector-v2/source/DB2.md | 155 ++++++++++++++++++++++ docs/en/connector-v2/source/Vertica.md | 157 ++++++++++++++++++++++ 4 files changed, 656 insertions(+) create mode 100644 docs/en/connector-v2/sink/DB2.md create mode 100644 docs/en/connector-v2/sink/Vertica.md create mode 100644 docs/en/connector-v2/source/DB2.md create mode 100644 docs/en/connector-v2/source/Vertica.md diff --git a/docs/en/connector-v2/sink/DB2.md b/docs/en/connector-v2/sink/DB2.md new file mode 100644 index 000000000000..8f5a7285e35d --- /dev/null +++ b/docs/en/connector-v2/sink/DB2.md @@ -0,0 +1,171 @@ +# DB2 + +> JDBC DB2 Sink Connector + +## Support Those Engines + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## Key Features + +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [cdc](../../concept/connector-v2-features.md) + +> Use `Xa transactions` to ensure `exactly-once`. So only support `exactly-once` for the database which is +> support `Xa transactions`. You can set `is_exactly_once=true` to enable it. + +## Description + +Write data through jdbc. Support Batch mode and Streaming mode, support concurrent writing, support exactly-once +semantics (using XA transaction guarantee). + +## Supported DataSource Info + +| Datasource | Supported Versions | Driver | Url | Maven | +|------------|----------------------------------------------------------|--------------------------------|-----------------------------------|-----------------------------------------------------------------------| +| DB2 | Different dependency version has different driver class. | com.ibm.db2.jdbc.app.DB2Driver | jdbc:db2://127.0.0.1:50000/dbname | [Download](https://mvnrepository.com/artifact/com.ibm.db2.jcc/db2jcc) | + +## Database Dependency + +> Please download the support list corresponding to 'Maven' and copy it to the '$SEATNUNNEL_HOME/plugins/jdbc/lib/' working directory
+> For example DB2 datasource: cp db2-connector-java-xxx.jar $SEATNUNNEL_HOME/plugins/jdbc/lib/ + +## Data Type Mapping + +| DB2 Data type | SeaTunnel Data type | +|------------------------------------------------------------------------------------------------------|---------------------|---| +| BOOLEAN | BOOLEAN | +| SMALLINT | SHORT | +| INT
INTEGER
| INTEGER | +| BIGINT | LONG | +| DECIMAL
DEC
NUMERIC
NUM | DECIMAL(38,18) | +| REAL | FLOAT | +| FLOAT
DOUBLE
DOUBLE PRECISION
DECFLOAT | DOUBLE | +| CHAR
VARCHAR
LONG VARCHAR
CLOB
GRAPHIC
VARGRAPHIC
LONG VARGRAPHIC
DBCLOB | STRING | +| BLOB | BYTES | +| DATE | DATE | +| TIME | TIME | +| TIMESTAMP | TIMESTAMP | +| ROWID
XML | Not supported yet | + +## 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 `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, 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 | + +### Tips + +> If partition_column is not set, it will run in single concurrency, and if partition_column is set, it will be executed in parallel according to the concurrency of tasks. + +## Task Example + +### 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 DB2. 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:db2://127.0.0.1:50000/dbname" + driver = "com.ibm.db2.jdbc.app.DB2Driver" + 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 +} +``` + +### Generate Sink SQL + +> This example not need to write complex sql statements, you can configure the database name table name to automatically generate add statements for you + +``` +sink { + jdbc { + url = "jdbc:db2://127.0.0.1:50000/dbname" + driver = "com.ibm.db2.jdbc.app.DB2Driver" + user = "root" + password = "123456" + # Automatically generate sql statements based on database table names + generate_sink_sql = true + database = test + table = test_table + } +} +``` + +### Exactly-once : + +> For accurate write scene we guarantee accurate once + +``` +sink { + jdbc { + url = "jdbc:db2://127.0.0.1:50000/dbname" + driver = "com.ibm.db2.jdbc.app.DB2Driver" + + max_retries = 0 + user = "root" + password = "123456" + query = "insert into test_table(name,age) values(?,?)" + + is_exactly_once = "true" + + xa_data_source_class_name = "com.db2.cj.jdbc.Db2XADataSource" + } +} +``` + diff --git a/docs/en/connector-v2/sink/Vertica.md b/docs/en/connector-v2/sink/Vertica.md new file mode 100644 index 000000000000..0db8571d55f2 --- /dev/null +++ b/docs/en/connector-v2/sink/Vertica.md @@ -0,0 +1,173 @@ +# Vertica + +> JDBC Vertica Sink Connector + +## Support Those Engines + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## Key Features + +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [cdc](../../concept/connector-v2-features.md) + +> Use `Xa transactions` to ensure `exactly-once`. So only support `exactly-once` for the database which is +> support `Xa transactions`. You can set `is_exactly_once=true` to enable it. + +## Description + +Write data through jdbc. Support Batch mode and Streaming mode, support concurrent writing, support exactly-once +semantics (using XA transaction guarantee). + +## Supported DataSource Info + +| Datasource | Supported Versions | Driver | Url | Maven | +|------------|----------------------------------------------------------|-------------------------|---------------------------------------|----------------------------------------------------------------------| +| Vertica | Different dependency version has different driver class. | com.vertica.jdbc.Driver | jdbc:vertica://localhost:5433/vertica | [Download](https://www.vertica.com/download/vertica/client-drivers/) | + +## Database Dependency + +> Please download the support list corresponding to 'Maven' and copy it to the '$SEATNUNNEL_HOME/plugins/jdbc/lib/' working directory
+> For example Vertica datasource: cp vertica-jdbc-xxx.jar $SEATNUNNEL_HOME/plugins/jdbc/lib/ + +## Data Type Mapping + +| Vertica Data type | SeaTunnel Data type | +|-----------------------------------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------| +| BIT(1)
INT UNSIGNED | BOOLEAN | +| TINYINT
TINYINT UNSIGNED
SMALLINT
SMALLINT UNSIGNED
MEDIUMINT
MEDIUMINT UNSIGNED
INT
INTEGER
YEAR | INT | +| INT UNSIGNED
INTEGER UNSIGNED
BIGINT | BIGINT | +| BIGINT UNSIGNED | DECIMAL(20,0) | +| DECIMAL(x,y)(Get the designated column's specified column size.<38) | DECIMAL(x,y) | +| DECIMAL(x,y)(Get the designated column's specified column size.>38) | DECIMAL(38,18) | +| DECIMAL UNSIGNED | DECIMAL((Get the designated column's specified column size)+1,
(Gets the designated column's number of digits to right of the decimal point.))) | +| FLOAT
FLOAT UNSIGNED | FLOAT | +| DOUBLE
DOUBLE UNSIGNED | DOUBLE | +| CHAR
VARCHAR
TINYTEXT
MEDIUMTEXT
TEXT
LONGTEXT
JSON | STRING | +| DATE | DATE | +| TIME | TIME | +| DATETIME
TIMESTAMP | TIMESTAMP | +| TINYBLOB
MEDIUMBLOB
BLOB
LONGBLOB
BINARY
VARBINAR
BIT(n) | BYTES | +| GEOMETRY
UNKNOWN | Not supported yet | + +## 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 `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, 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 | + +### Tips + +> If partition_column is not set, it will run in single concurrency, and if partition_column is set, it will be executed in parallel according to the concurrency of tasks. + +## Task Example + +### 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 vertical. 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:vertica://localhost:5433/vertica" + driver = "com.vertica.jdbc.Driver" + 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 +} +``` + +### Generate Sink SQL + +> This example not need to write complex sql statements, you can configure the database name table name to automatically generate add statements for you + +``` +sink { + jdbc { + url = "jdbc:vertica://localhost:5433/vertica" + driver = "com.vertica.jdbc.Driver" + user = "root" + password = "123456" + # Automatically generate sql statements based on database table names + generate_sink_sql = true + database = test + table = test_table + } +} +``` + +### Exactly-once : + +> For accurate write scene we guarantee accurate once + +``` +sink { + jdbc { + url = "jdbc:vertica://localhost:5433/vertica" + driver = "com.vertica.jdbc.Driver" + + max_retries = 0 + user = "root" + password = "123456" + query = "insert into test_table(name,age) values(?,?)" + + is_exactly_once = "true" + + xa_data_source_class_name = "com.vertical.cj.jdbc.VerticalXADataSource" + } +} +``` + diff --git a/docs/en/connector-v2/source/DB2.md b/docs/en/connector-v2/source/DB2.md new file mode 100644 index 000000000000..7ea91b7165c7 --- /dev/null +++ b/docs/en/connector-v2/source/DB2.md @@ -0,0 +1,155 @@ +# DB2 + +> JDBC DB2 Source Connector + +## Support Those Engines + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## Key Features + +- [x] [batch](../../concept/connector-v2-features.md) +- [ ] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [x] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) + +> supports query SQL and can achieve projection effect. + +## Description + +Read external data source data through JDBC. + +## Supported DataSource Info + +| Datasource | Supported versions | Driver | Url | Maven | +|------------|----------------------------------------------------------|--------------------------------|-----------------------------------|-----------------------------------------------------------------------| +| DB2 | Different dependency version has different driver class. | com.ibm.db2.jdbc.app.DB2Driver | jdbc:db2://127.0.0.1:50000/dbname | [Download](https://mvnrepository.com/artifact/com.ibm.db2.jcc/db2jcc) | + +## Database Dependency + +> Please download the support list corresponding to 'Maven' and copy it to the '$SEATNUNNEL_HOME/plugins/jdbc/lib/' working directory
+> For example DB2 datasource: cp db2-connector-java-xxx.jar $SEATNUNNEL_HOME/plugins/jdbc/lib/ + +## Data Type Mapping + +| DB2 Data type | SeaTunnel Data type | +|------------------------------------------------------------------------------------------------------|---------------------|---| +| BOOLEAN | BOOLEAN | +| SMALLINT | SHORT | +| INT
INTEGER
| INTEGER | +| BIGINT | LONG | +| DECIMAL
DEC
NUMERIC
NUM | DECIMAL(38,18) | +| REAL | FLOAT | +| FLOAT
DOUBLE
DOUBLE PRECISION
DECFLOAT | DOUBLE | +| CHAR
VARCHAR
LONG VARCHAR
CLOB
GRAPHIC
VARGRAPHIC
LONG VARGRAPHIC
DBCLOB | STRING | +| BLOB | BYTES | +| DATE | DATE | +| TIME | TIME | +| TIMESTAMP | TIMESTAMP | +| ROWID
XML | Not supported yet | + +## Source 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 | Yes | - | Query statement | +| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete | +| partition_column | String | No | - | The column name for parallelism's partition, only support numeric type,Only support numeric type primary key, and only can config one column. | +| partition_lower_bound | Long | No | - | The partition_column min value for scan, if not set SeaTunnel will query database get min value. | +| partition_upper_bound | Long | 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. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | + +### Tips + +> If partition_column is not set, it will run in single concurrency, and if partition_column is set, it will be executed in parallel according to the concurrency of tasks. + +## Task Example + +### Simple: + +> This example queries type_bin 'table' 16 data in your test "database" in single parallel and queries all of its fields. You can also specify which fields to query for final output to the console. + +``` +# Defining the runtime environment +env { + # You can set flink configuration here + execution.parallelism = 2 + job.mode = "BATCH" +} +source{ + Jdbc { + url = "jdbc:db2://127.0.0.1:50000/dbname" + driver = "com.ibm.db2.jdbc.app.DB2Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + query = "select * from table_xxx" + } +} + +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/transform-v2/sql +} + +sink { + Console {} +} +``` + +### Parallel: + +> Read your query table in parallel with the shard field you configured and the shard data You can do this if you want to read the whole table + +``` +source { + Jdbc { + url = "jdbc:db2://127.0.0.1:50000/dbname" + driver = "com.ibm.db2.jdbc.app.DB2Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + # Define query logic as required + query = "select * from type_bin" + # Parallel sharding reads fields + partition_column = "id" + # Number of fragments + partition_num = 10 + } +} +``` + +### Parallel Boundary: + +> It is more efficient to specify the data within the upper and lower bounds of the query It is more efficient to read your data source according to the upper and lower boundaries you configured + +``` +source { + Jdbc { + url = "jdbc:db2://127.0.0.1:50000/dbname" + driver = "com.ibm.db2.jdbc.app.DB2Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + # Define query logic as required + query = "select * from type_bin" + partition_column = "id" + # Read start boundary + partition_lower_bound = 1 + # Read end boundary + partition_upper_bound = 500 + partition_num = 10 + } +} +``` + diff --git a/docs/en/connector-v2/source/Vertica.md b/docs/en/connector-v2/source/Vertica.md new file mode 100644 index 000000000000..66f18e7a4ed4 --- /dev/null +++ b/docs/en/connector-v2/source/Vertica.md @@ -0,0 +1,157 @@ +# Vertica + +> JDBC Vertica Source Connector + +## Support Those Engines + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## Key Features + +- [x] [batch](../../concept/connector-v2-features.md) +- [ ] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [x] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) + +> supports query SQL and can achieve projection effect. + +## Description + +Read external data source data through JDBC. + +## Supported DataSource Info + +| Datasource | Supported versions | Driver | Url | Maven | +|------------|----------------------------------------------------------|-------------------------|---------------------------------------|----------------------------------------------------------------------| +| Vertica | Different dependency version has different driver class. | com.vertica.jdbc.Driver | jdbc:vertica://localhost:5433/vertica | [Download](https://www.vertica.com/download/vertica/client-drivers/) | + +## Database Dependency + +> Please download the support list corresponding to 'Maven' and copy it to the '$SEATNUNNEL_HOME/plugins/jdbc/lib/' working directory
+> For example Vertica datasource: cp vertica-jdbc-xxx.jar $SEATNUNNEL_HOME/plugins/jdbc/lib/ + +## Data Type Mapping + +| Vertical Data type | SeaTunnel Data type | +|-----------------------------------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------| +| BIT | BOOLEAN | +| TINYINT
TINYINT UNSIGNED
SMALLINT
SMALLINT UNSIGNED
MEDIUMINT
MEDIUMINT UNSIGNED
INT
INTEGER
YEAR | INT | +| INT UNSIGNED
INTEGER UNSIGNED
BIGINT | LONG | +| BIGINT UNSIGNED | DECIMAL(20,0) | +| DECIMAL(x,y)(Get the designated column's specified column size.<38) | DECIMAL(x,y) | +| DECIMAL(x,y)(Get the designated column's specified column size.>38) | DECIMAL(38,18) | +| DECIMAL UNSIGNED | DECIMAL((Get the designated column's specified column size)+1,
(Gets the designated column's number of digits to right of the decimal point.))) | +| FLOAT
FLOAT UNSIGNED | FLOAT | +| DOUBLE
DOUBLE UNSIGNED | DOUBLE | +| CHAR
VARCHAR
TINYTEXT
MEDIUMTEXT
TEXT
LONGTEXT
JSON | STRING | +| DATE | DATE | +| TIME | TIME | +| DATETIME
TIMESTAMP | TIMESTAMP | +| TINYBLOB
MEDIUMBLOB
BLOB
LONGBLOB
BINARY
VARBINAR
BIT(n) | BYTES | +| GEOMETRY
UNKNOWN | Not supported yet | + +## Source 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 Vertica the value is `com.vertica.jdbc.Driver`. | +| user | String | No | - | Connection instance user name | +| password | String | No | - | Connection instance password | +| query | String | Yes | - | Query statement | +| connection_check_timeout_sec | Int | No | 30 | The time in seconds to wait for the database operation used to validate the connection to complete | +| partition_column | String | No | - | The column name for parallelism's partition, only support numeric type,Only support numeric type primary key, and only can config one column. | +| partition_lower_bound | Long | No | - | The partition_column min value for scan, if not set SeaTunnel will query database get min value. | +| partition_upper_bound | Long | 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. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | + +### Tips + +> If partition_column is not set, it will run in single concurrency, and if partition_column is set, it will be executed in parallel according to the concurrency of tasks. + +## Task Example + +### Simple: + +> This example queries type_bin 'table' 16 data in your test "database" in single parallel and queries all of its fields. You can also specify which fields to query for final output to the console. + +``` +# Defining the runtime environment +env { + # You can set flink configuration here + execution.parallelism = 2 + job.mode = "BATCH" +} +source{ + Jdbc { + url = "jdbc:vertica://localhost:5433/vertica" + driver = "com.vertica.jdbc.Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + query = "select * from type_bin limit 16" + } +} + +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/transform-v2/sql +} + +sink { + Console {} +} +``` + +### Parallel: + +> Read your query table in parallel with the shard field you configured and the shard data You can do this if you want to read the whole table + +``` +source { + Jdbc { + url = "jdbc:vertica://localhost:5433/vertica" + driver = "com.vertica.jdbc.Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + # Define query logic as required + query = "select * from type_bin" + # Parallel sharding reads fields + partition_column = "id" + # Number of fragments + partition_num = 10 + } +} +``` + +### Parallel Boundary: + +> It is more efficient to specify the data within the upper and lower bounds of the query It is more efficient to read your data source according to the upper and lower boundaries you configured + +``` +source { + Jdbc { + url = "jdbc:vertica://localhost:5433/vertica" + driver = "com.vertica.jdbc.Driver" + connection_check_timeout_sec = 100 + user = "root" + password = "123456" + # Define query logic as required + query = "select * from type_bin" + partition_column = "id" + # Read start boundary + partition_lower_bound = 1 + # Read end boundary + partition_upper_bound = 500 + partition_num = 10 + } +} +``` + From 70ec3a3608d91359c89cbb9c9da89b6f041ef7d7 Mon Sep 17 00:00:00 2001 From: monster <60029759+MonsterChenzhuo@users.noreply.github.com> Date: Wed, 19 Jul 2023 17:25:45 +0800 Subject: [PATCH 04/43] [Improve][Docs][Clickhouse] Reconstruct the clickhouse connector doc (#5085) --------- Co-authored-by: chenzy15 --- docs/en/connector-v2/sink/Clickhouse.md | 207 +++++++++++----------- docs/en/connector-v2/source/Clickhouse.md | 129 +++++++------- 2 files changed, 168 insertions(+), 168 deletions(-) diff --git a/docs/en/connector-v2/sink/Clickhouse.md b/docs/en/connector-v2/sink/Clickhouse.md index 7c4bab991ba4..27bf274c77fe 100644 --- a/docs/en/connector-v2/sink/Clickhouse.md +++ b/docs/en/connector-v2/sink/Clickhouse.md @@ -2,95 +2,110 @@ > Clickhouse sink connector -## Description +## Support Those Engines -Used to write data to Clickhouse. +> Spark
+> Flink
+> SeaTunnel Zeta
-## Key features +## Key Features - [ ] [exactly-once](../../concept/connector-v2-features.md) - -The Clickhouse sink plug-in can achieve accuracy once by implementing idempotent writing, and needs to cooperate with aggregatingmergetree and other engines that support deduplication. - - [x] [cdc](../../concept/connector-v2-features.md) -## Options - -| name | type | required | default value | -|---------------------------------------|---------|----------|---------------| -| host | string | yes | - | -| database | string | yes | - | -| table | string | yes | - | -| username | string | yes | - | -| password | string | yes | - | -| clickhouse.config | map | no | | -| bulk_size | string | no | 20000 | -| split_mode | string | no | false | -| sharding_key | string | no | - | -| primary_key | string | no | - | -| support_upsert | boolean | no | false | -| allow_experimental_lightweight_delete | boolean | no | false | -| common-options | | no | - | - -### host [string] - -`ClickHouse` cluster address, the format is `host:port` , allowing multiple `hosts` to be specified. Such as `"host1:8123,host2:8123"` . - -### database [string] - -The `ClickHouse` database - -### table [string] - -The table name - -### username [string] - -`ClickHouse` user username - -### password [string] - -`ClickHouse` user password - -### clickhouse.config [map] - -In addition to the above mandatory parameters that must be specified by `clickhouse-jdbc` , users can also specify multiple optional parameters, which cover all the [parameters](https://github.com/ClickHouse/clickhouse-jdbc/tree/master/clickhouse-client#configuration) provided by `clickhouse-jdbc` . - -### bulk_size [number] - -The number of rows written through [Clickhouse-jdbc](https://github.com/ClickHouse/clickhouse-jdbc) each time, the `default is 20000`, if checkpoints are enabled, writing will also occur at the times when the checkpoints are satisfied . - -### split_mode [boolean] - -This mode only support clickhouse table which engine is 'Distributed'.And `internal_replication` option -should be `true`. They will split distributed table data in seatunnel and perform write directly on each shard. The shard weight define is clickhouse will be -counted. - -### sharding_key [string] +> The Clickhouse sink plug-in can achieve accuracy once by implementing idempotent writing, and needs to cooperate with aggregatingmergetree and other engines that support deduplication. -When use split_mode, which node to send data to is a problem, the default is random selection, but the -'sharding_key' parameter can be used to specify the field for the sharding algorithm. This option only -worked when 'split_mode' is true. - -### primary_key [string] - -Mark the primary key column from clickhouse table, and based on primary key execute INSERT/UPDATE/DELETE to clickhouse table - -### support_upsert [boolean] +## Description -Support upsert row by query primary key +Used to write data to Clickhouse. -### allow_experimental_lightweight_delete [boolean] +## Supported DataSource Info + +In order to use the Clickhouse connector, the following dependencies are required. +They can be downloaded via install-plugin.sh or from the Maven central repository. + +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------------------------------------| +| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-clickhouse) | + +## Data Type Mapping + +| SeaTunnel Data type | Clickhouse Data type | +|---------------------|-----------------------------------------------------------------------------------------------------------------------------------------------| +| STRING | String / Int128 / UInt128 / Int256 / UInt256 / Point / Ring / Polygon MultiPolygon | +| INT | Int8 / UInt8 / Int16 / UInt16 / Int32 | +| BIGINT | UInt64 / Int64 / IntervalYear / IntervalQuarter / IntervalMonth / IntervalWeek / IntervalDay / IntervalHour / IntervalMinute / IntervalSecond | +| DOUBLE | Float64 | +| DECIMAL | Decimal | +| FLOAT | Float32 | +| DATE | Date | +| TIME | DateTime | +| ARRAY | Array | +| MAP | Map | + +## Sink Options + +| Name | Type | Required | Default | Description | +|---------------------------------------|---------|----------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| host | String | Yes | - | `ClickHouse` cluster address, the format is `host:port` , allowing multiple `hosts` to be specified. Such as `"host1:8123,host2:8123"`. | +| database | String | Yes | - | The `ClickHouse` database. | +| table | String | Yes | - | The table name. | +| username | String | Yes | - | `ClickHouse` user username. | +| password | String | Yes | - | `ClickHouse` user password. | +| clickhouse.config | Map | No | | In addition to the above mandatory parameters that must be specified by `clickhouse-jdbc` , users can also specify multiple optional parameters, which cover all the [parameters](https://github.com/ClickHouse/clickhouse-jdbc/tree/master/clickhouse-client#configuration) provided by `clickhouse-jdbc`. | +| bulk_size | String | No | 20000 | The number of rows written through [Clickhouse-jdbc](https://github.com/ClickHouse/clickhouse-jdbc) each time, the `default is 20000`. | +| split_mode | String | No | false | This mode only support clickhouse table which engine is 'Distributed'.And `internal_replication` option-should be `true`.They will split distributed table data in seatunnel and perform write directly on each shard. The shard weight define is clickhouse will counted. | +| sharding_key | String | No | - | When use split_mode, which node to send data to is a problem, the default is random selection, but the 'sharding_key' parameter can be used to specify the field for the sharding algorithm. This option only worked when 'split_mode' is true. | +| primary_key | String | No | - | Mark the primary key column from clickhouse table, and based on primary key execute INSERT/UPDATE/DELETE to clickhouse table. | +| support_upsert | Boolean | No | false | Support upsert row by query primary key. | +| allow_experimental_lightweight_delete | Boolean | No | false | Allow experimental lightweight delete based on `*MergeTree` table engine. | +| common-options | | No | - | Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details. | + +## How to Create a Clickhouse Data Synchronization Jobs + +The following example demonstrates how to create a data synchronization job that writes randomly generated data to a Clickhouse database: + +```bash +# Set the basic configuration of the task to be performed +env { + execution.parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 1000 +} -Allow experimental lightweight delete based on `*MergeTree` table engine +source { + FakeSource { + row.num = 2 + bigint.min = 0 + bigint.max = 10000000 + split.num = 1 + split.read-interval = 300 + schema { + fields { + c_bigint = bigint + } + } + } +} -### common options +sink { + Clickhouse { + host = "127.0.0.1:9092" + database = "default" + table = "test" + username = "xxxxx" + password = "xxxxx" + } +} +``` -Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details +### Tips -## Examples +> 1.[SeaTunnel Deployment Document](../../start-v2/locally/deployment.md).
+> 2.The table to be written to needs to be created in advance before synchronization.
+> 3.When sink is writing to the ClickHouse table, you don't need to set its schema because the connector will query ClickHouse for the current table's schema information before writing.
-Simple +## Clickhouse Sink Config ```hocon sink { @@ -98,9 +113,9 @@ sink { host = "localhost:8123" database = "default" table = "fake_all" - username = "default" - password = "" - clickhouse.confg = { + username = "xxxxx" + password = "xxxxx" + clickhouse.config = { max_rows_to_read = "100" read_overflow_mode = "throw" } @@ -108,7 +123,7 @@ sink { } ``` -Split mode +## Split Mode ```hocon sink { @@ -116,8 +131,8 @@ sink { host = "localhost:8123" database = "default" table = "fake_all" - username = "default" - password = "" + username = "xxxxx" + password = "xxxxx" # split mode options split_mode = true @@ -126,7 +141,7 @@ sink { } ``` -CDC(Change data capture) +## CDC(Change data capture) Sink ```hocon sink { @@ -134,8 +149,8 @@ sink { host = "localhost:8123" database = "default" table = "fake_all" - username = "default" - password = "" + username = "xxxxx" + password = "xxxxx" # cdc options primary_key = "id" @@ -144,7 +159,7 @@ sink { } ``` -CDC(Change data capture) for *MergeTree engine +## CDC(Change data capture) for *MergeTree engine ```hocon sink { @@ -152,8 +167,8 @@ sink { host = "localhost:8123" database = "default" table = "fake_all" - username = "default" - password = "" + username = "xxxxx" + password = "xxxxx" # cdc options primary_key = "id" @@ -163,21 +178,3 @@ sink { } ``` -## Changelog - -### 2.2.0-beta 2022-09-26 - -- Add ClickHouse Sink Connector - -### 2.3.0-beta 2022-10-20 - -- [Improve] Clickhouse Support Int128,Int256 Type ([3067](https://github.com/apache/seatunnel/pull/3067)) - -### next version - -- [Improve] Clickhouse Sink support nest type and array type([3047](https://github.com/apache/seatunnel/pull/3047)) -- [Improve] Clickhouse Sink support geo type([3141](https://github.com/apache/seatunnel/pull/3141)) -- [Feature] Support CDC write DELETE/UPDATE/INSERT events ([3653](https://github.com/apache/seatunnel/pull/3653)) -- [Improve] Remove Clickhouse Fields Config ([3826](https://github.com/apache/seatunnel/pull/3826)) -- [Improve] Change Connector Custom Config Prefix To Map [3719](https://github.com/apache/seatunnel/pull/3719) - diff --git a/docs/en/connector-v2/source/Clickhouse.md b/docs/en/connector-v2/source/Clickhouse.md index 07384875cb0d..7596bf72a8f0 100644 --- a/docs/en/connector-v2/source/Clickhouse.md +++ b/docs/en/connector-v2/source/Clickhouse.md @@ -2,93 +2,96 @@ > Clickhouse source connector -## Description +## Support Those Engines -Used to read data from Clickhouse. +> Spark
+> Flink
+> SeaTunnel Zeta
-## Key features +## Key Features - [x] [batch](../../concept/connector-v2-features.md) - [ ] [stream](../../concept/connector-v2-features.md) - [ ] [exactly-once](../../concept/connector-v2-features.md) - [x] [column projection](../../concept/connector-v2-features.md) - -supports query SQL and can achieve projection effect. - - [ ] [parallelism](../../concept/connector-v2-features.md) - [ ] [support user-defined split](../../concept/connector-v2-features.md) -## Options - -| name | type | required | default value | -|------------------|--------|----------|------------------------| -| host | string | yes | - | -| database | string | yes | - | -| sql | string | yes | - | -| username | string | yes | - | -| password | string | yes | - | -| server_time_zone | string | no | ZoneId.systemDefault() | -| common-options | | no | - | - -### host [string] - -`ClickHouse` cluster address, the format is `host:port` , allowing multiple `hosts` to be specified. Such as `"host1:8123,host2:8123"` . - -### database [string] - -The `ClickHouse` database - -### sql [string] - -The query sql used to search data though Clickhouse server - -### username [string] - -`ClickHouse` user username - -### password [string] - -`ClickHouse` user password +> supports query SQL and can achieve projection effect. -### server_time_zone [string] - -The session time zone in database server. If not set, then ZoneId.systemDefault() is used to determine the server time zone. - -### common options +## Description -Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details +Used to read data from Clickhouse. -## Examples +## Supported DataSource Info + +In order to use the Clickhouse connector, the following dependencies are required. +They can be downloaded via install-plugin.sh or from the Maven central repository. + +| Datasource | Supported Versions | Dependency | +|------------|--------------------|------------------------------------------------------------------------------------------------------------------| +| Clickhouse | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-clickhouse) | + +## Data Type Mapping + +| Clickhouse Data type | SeaTunnel Data type | +|-----------------------------------------------------------------------------------------------------------------------------------------------|---------------------| +| String / Int128 / UInt128 / Int256 / UInt256 / Point / Ring / Polygon MultiPolygon | STRING | +| Int8 / UInt8 / Int16 / UInt16 / Int32 | INT | +| UInt64 / Int64 / IntervalYear / IntervalQuarter / IntervalMonth / IntervalWeek / IntervalDay / IntervalHour / IntervalMinute / IntervalSecond | BIGINT | +| Float64 | DOUBLE | +| Decimal | DECIMAL | +| Float32 | FLOAT | +| Date | DATE | +| DateTime | TIME | +| Array | ARRAY | +| Map | MAP | + +## Source Options + +| Name | Type | Required | Default | Description | +|------------------|--------|----------|------------------------|------------------------------------------------------------------------------------------------------------------------------------------| +| host | String | Yes | - | `ClickHouse` cluster address, the format is `host:port` , allowing multiple `hosts` to be specified. Such as `"host1:8123,host2:8123"` . | +| database | String | Yes | - | The `ClickHouse` database. | +| sql | String | Yes | - | The query sql used to search data though Clickhouse server. | +| username | String | Yes | - | `ClickHouse` user username. | +| password | String | Yes | - | `ClickHouse` user password. | +| server_time_zone | String | No | ZoneId.systemDefault() | The session time zone in database server. If not set, then ZoneId.systemDefault() is used to determine the server time zone. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. | + +## How to Create a Clickhouse Data Synchronization Jobs + +The following example demonstrates how to create a data synchronization job that reads data from Clickhouse and prints it on the local client: + +```bash +# Set the basic configuration of the task to be performed +env { + execution.parallelism = 1 + job.mode = "BATCH" +} -```hocon +# Create a source to connect to Clickhouse source { - Clickhouse { host = "localhost:8123" database = "default" sql = "select * from test where age = 20 limit 100" - username = "default" - password = "" + username = "xxxxx" + password = "xxxxx" server_time_zone = "UTC" result_table_name = "test" } - } -``` - -## Changelog -### 2.2.0-beta 2022-09-26 - -- Add ClickHouse Source Connector - -### 2.3.0-beta 2022-10-20 - -- [Improve] Clickhouse Source random use host when config multi-host ([3108](https://github.com/apache/seatunnel/pull/3108)) - -### next version +# Console printing of the read Clickhouse data +sink { + Console { + parallelism = 1 + } +} +``` -- [Improve] Clickhouse Source support nest type and array type([3047](https://github.com/apache/seatunnel/pull/3047)) +### Tips -- [Improve] Clickhouse Source support geo type([3141](https://github.com/apache/seatunnel/pull/3141)) +> 1.[SeaTunnel Deployment Document](../../start-v2/locally/deployment.md). From 2e92c987e4d4a9e7bf2da520ad73c375d23e3570 Mon Sep 17 00:00:00 2001 From: Eric Date: Wed, 19 Jul 2023 18:56:57 +0800 Subject: [PATCH 05/43] [Pom]update version to 2.3.3-SNAPSHOT (#5043) * update version to 2.3.3-SNAPSHOT * update dependency version in know dependencies file * Add logs to find job restore from master active switch error --- pom.xml | 2 +- .../server/checkpoint/CheckpointManager.java | 4 ++++ .../server/dag/physical/PhysicalVertex.java | 2 +- .../engine/server/dag/physical/SubPlan.java | 15 ++++++++++++--- tools/dependencies/known-dependencies.txt | 4 ++-- 5 files changed, 20 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index 51b03a26d5bb..7dce624be315 100644 --- a/pom.xml +++ b/pom.xml @@ -55,7 +55,7 @@ - 2.3.2-SNAPSHOT + 2.3.3-SNAPSHOT 2.1.1 UTF-8 1.8 diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java index 9f9649f03a10..f34ae2f6a0ad 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java @@ -47,6 +47,7 @@ import com.hazelcast.spi.impl.operationservice.impl.InvocationFuture; import lombok.extern.slf4j.Slf4j; +import java.util.Arrays; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -167,6 +168,9 @@ public PassiveCompletableFuture triggerSavepoint(int pipeli } public void reportedPipelineRunning(int pipelineId, boolean alreadyStarted) { + log.info( + "reported pipeline running stack: " + + Arrays.toString(Thread.currentThread().getStackTrace())); getCheckpointCoordinator(pipelineId).restoreCoordinator(alreadyStarted); } diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java index 65666413548e..3c840a269ad0 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/PhysicalVertex.java @@ -394,7 +394,7 @@ private boolean turnToEndState(@NonNull ExecutionState endState) { public boolean updateTaskState( @NonNull ExecutionState current, @NonNull ExecutionState targetState) { synchronized (this) { - LOGGER.fine( + LOGGER.info( String.format( "Try to update the task %s state from %s to %s", taskFullName, current, targetState)); diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/SubPlan.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/SubPlan.java index bc9e3e2aaefe..83dd4e9d0f22 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/SubPlan.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/dag/physical/SubPlan.java @@ -53,7 +53,7 @@ public class SubPlan { private static final ILogger LOGGER = Logger.getLogger(SubPlan.class); /** The max num pipeline can restore. */ - public static final int PIPELINE_MAX_RESTORE_NUM = 2; // TODO should set by config + public static final int PIPELINE_MAX_RESTORE_NUM = 3; // TODO should set by config private final List physicalVertexList; @@ -332,6 +332,9 @@ private void turnToEndState(@NonNull PipelineStatus endState) throws Exception { exception -> ExceptionUtil.isOperationNeedRetryException(exception), Constant.OPERATION_RETRY_SLEEP)); this.currPipelineStatus = endState; + LOGGER.info( + String.format( + "%s turn to end state %s.", pipelineFullName, currPipelineStatus)); } } @@ -511,11 +514,17 @@ private void resetPipelineState() throws Exception { LOGGER.severe(message); throw new IllegalStateException(message); } - + LOGGER.info( + String.format( + "Reset pipeline %s state to %s", + getPipelineFullName(), PipelineStatus.CREATED)); updateStateTimestamps(PipelineStatus.CREATED); runningJobStateIMap.set(pipelineLocation, PipelineStatus.CREATED); this.currPipelineStatus = PipelineStatus.CREATED; - ; + LOGGER.info( + String.format( + "Reset pipeline %s state to %s complete", + getPipelineFullName(), PipelineStatus.CREATED)); return null; }, new RetryUtils.RetryMaterial( diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 3a1e736b68b5..70bbd1c0df53 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -22,8 +22,8 @@ protostuff-collectionschema-1.8.0.jar protostuff-core-1.8.0.jar protostuff-runtime-1.8.0.jar scala-library-2.11.12.jar -seatunnel-jackson-2.3.2-SNAPSHOT-optional.jar -seatunnel-guava-2.3.2-SNAPSHOT-optional.jar +seatunnel-jackson-2.3.3-SNAPSHOT-optional.jar +seatunnel-guava-2.3.3-SNAPSHOT-optional.jar slf4j-api-1.7.25.jar jsqlparser-4.5.jar animal-sniffer-annotations-1.17.jar From 86b1b7e31a351dafc31404f1b3262f0bc0ee70be Mon Sep 17 00:00:00 2001 From: Carl-Zhou-CN <67902676+Carl-Zhou-CN@users.noreply.github.com> Date: Thu, 20 Jul 2023 12:59:37 +0800 Subject: [PATCH 06/43] [Feature][Connector-V2][mysql cdc] Conversion of tinyint(1) to bool is supported (#5105) Co-authored-by: zhouyao --- .../connectors/seatunnel/cdc/mysql/utils/MySqlTypeUtils.java | 1 + 1 file changed, 1 insertion(+) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlTypeUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlTypeUtils.java index 01760bfc9f61..267476b3ffea 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlTypeUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/utils/MySqlTypeUtils.java @@ -87,6 +87,7 @@ public static SeaTunnelDataType convertFromColumn(Column column) { case MYSQL_BIT: return BasicType.BOOLEAN_TYPE; case MYSQL_TINYINT: + return column.length() == 1 ? BasicType.BOOLEAN_TYPE : BasicType.INT_TYPE; case MYSQL_TINYINT_UNSIGNED: case MYSQL_SMALLINT: case MYSQL_SMALLINT_UNSIGNED: From 02114db626a290502d33797dfbe9a11ee5b52b6f Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Thu, 20 Jul 2023 15:25:34 +0800 Subject: [PATCH 07/43] [Improve][Zeta] Add sleep for Task to reduce CPU cost (#5117) --- .../seatunnel/engine/server/task/AbstractTask.java | 4 +--- .../engine/server/task/SeaTunnelSourceCollector.java | 12 ++++++------ .../server/task/SinkAggregatedCommitterTask.java | 8 +++++++- .../engine/server/task/flow/SourceFlowLifeCycle.java | 6 ++++-- 4 files changed, 18 insertions(+), 12 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/AbstractTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/AbstractTask.java index 776329716528..9f07428ff4ce 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/AbstractTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/AbstractTask.java @@ -81,9 +81,7 @@ public void init() throws Exception { } @NonNull @Override - public ProgressState call() throws Exception { - return progress.toState(); - } + public abstract ProgressState call() throws Exception; public TaskLocation getTaskLocation() { return this.taskLocation; diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java index 5cab2dd0b24a..2a77a49729fd 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelSourceCollector.java @@ -40,7 +40,7 @@ public class SeaTunnelSourceCollector implements Collector { private final Meter sourceReceivedQPS; - private volatile long rowCountThisPollNext; + private volatile boolean emptyThisPollNext; public SeaTunnelSourceCollector( Object checkpointLock, @@ -56,7 +56,7 @@ public SeaTunnelSourceCollector( public void collect(T row) { try { sendRecordToNext(new Record<>(row)); - rowCountThisPollNext++; + emptyThisPollNext = false; sourceReceivedCount.inc(); sourceReceivedQPS.markEvent(); } catch (IOException e) { @@ -69,12 +69,12 @@ public Object getCheckpointLock() { return checkpointLock; } - public long getRowCountThisPollNext() { - return this.rowCountThisPollNext; + public boolean isEmptyThisPollNext() { + return emptyThisPollNext; } - public void resetRowCountThisPollNext() { - this.rowCountThisPollNext = 0; + public void resetEmptyThisPollNext() { + this.emptyThisPollNext = true; } public void sendRecordToNext(Record record) throws IOException { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SinkAggregatedCommitterTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SinkAggregatedCommitterTask.java index 797033f8d2ef..a83f4bfb1dec 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SinkAggregatedCommitterTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SinkAggregatedCommitterTask.java @@ -68,7 +68,7 @@ public class SinkAggregatedCommitterTask private static final long serialVersionUID = 5906594537520393503L; - private SeaTunnelTaskState currState; + private volatile SeaTunnelTaskState currState; private final SinkAction sink; private final int maxWriterSize; @@ -138,16 +138,22 @@ protected void stateProcess() throws Exception { if (restoreComplete.isDone()) { currState = READY_START; reportTaskStatus(READY_START); + } else { + Thread.sleep(100); } break; case READY_START: if (startCalled) { currState = STARTING; + } else { + Thread.sleep(100); } break; case STARTING: if (receivedSinkWriter) { currState = RUNNING; + } else { + Thread.sleep(100); } break; case RUNNING: diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java index e4928343cf92..9ca01eba322d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SourceFlowLifeCycle.java @@ -133,11 +133,13 @@ public void close() throws IOException { public void collect() throws Exception { if (!prepareClose) { reader.pollNext(collector); - if (collector.getRowCountThisPollNext() == 0) { + if (collector.isEmptyThisPollNext()) { Thread.sleep(100); } else { - collector.resetRowCountThisPollNext(); + collector.resetEmptyThisPollNext(); } + } else { + Thread.sleep(100); } } From 5e8d982e25dd2a77f366baba0d5876d67e6fc2ad Mon Sep 17 00:00:00 2001 From: lihjChina <237206177@qq.com> Date: Thu, 20 Jul 2023 23:13:52 +0800 Subject: [PATCH 08/43] [Feature][JDBC Sink] Add DM upsert support (#5073) --------- Co-authored-by: David Zollo --- .../jdbc/internal/dialect/dm/DmdbDialect.java | 58 +++- .../seatunnel/jdbc/JdbcDmUpsetIT.java | 258 ++++++++++++++++++ .../jdbc_dm_source_and_dm_upset_sink.conf | 49 ++++ 3 files changed, 364 insertions(+), 1 deletion(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmUpsetIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/resources/jdbc_dm_source_and_dm_upset_sink.conf diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialect.java index c3a929be29c7..00845cf11a30 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/dm/DmdbDialect.java @@ -21,7 +21,10 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; +import java.util.Arrays; +import java.util.List; import java.util.Optional; +import java.util.stream.Collectors; public class DmdbDialect implements JdbcDialect { @@ -43,6 +46,59 @@ public JdbcDialectTypeMapper getJdbcDialectTypeMapper() { @Override public Optional getUpsertStatement( String database, String tableName, String[] fieldNames, String[] uniqueKeyFields) { - return Optional.empty(); + List nonUniqueKeyFields = + Arrays.stream(fieldNames) + .filter(fieldName -> !Arrays.asList(uniqueKeyFields).contains(fieldName)) + .collect(Collectors.toList()); + String valuesBinding = + Arrays.stream(fieldNames) + .map(fieldName -> ":" + fieldName + " " + quoteIdentifier(fieldName)) + .collect(Collectors.joining(", ")); + String usingClause = String.format("SELECT %s", valuesBinding); + String onConditions = + Arrays.stream(uniqueKeyFields) + .map( + fieldName -> + String.format( + "TARGET.%s=SOURCE.%s", + quoteIdentifier(fieldName), + quoteIdentifier(fieldName))) + .collect(Collectors.joining(" AND ")); + + String updateSetClause = + nonUniqueKeyFields.stream() + .map( + fieldName -> + String.format( + "TARGET.%s=SOURCE.%s", + quoteIdentifier(fieldName), + quoteIdentifier(fieldName))) + .collect(Collectors.joining(", ")); + + String insertFields = + Arrays.stream(fieldNames) + .map(this::quoteIdentifier) + .collect(Collectors.joining(", ")); + String insertValues = + Arrays.stream(fieldNames) + .map(fieldName -> "SOURCE." + quoteIdentifier(fieldName)) + .collect(Collectors.joining(", ")); + String upsertSQL = + String.format( + " MERGE INTO %s TARGET" + + " USING (%s) SOURCE" + + " ON (%s) " + + " WHEN MATCHED THEN" + + " UPDATE SET %s" + + " WHEN NOT MATCHED THEN" + + " INSERT (%s) VALUES (%s)", + tableIdentifier(database, tableName), + usingClause, + onConditions, + updateSetClause, + insertFields, + insertValues); + + return Optional.of(upsertSQL); } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmUpsetIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmUpsetIT.java new file mode 100644 index 000000000000..653394315481 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcDmUpsetIT.java @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +import org.apache.commons.lang3.tuple.Pair; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.utility.DockerLoggerFactory; + +import com.google.common.collect.Lists; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Statement; +import java.sql.Timestamp; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class JdbcDmUpsetIT extends AbstractJdbcIT { + + private static final String DM_IMAGE = "laglangyue/dmdb8"; + private static final String DM_CONTAINER_HOST = "e2e_dmdb_upset"; + + private static final String DM_DATABASE = "SYSDBA"; + private static final String DM_SOURCE = "E2E_TABLE_SOURCE_UPSET"; + private static final String DM_SINK = "E2E_TABLE_SINK_UPSET"; + private static final String DM_USERNAME = "SYSDBA"; + private static final String DM_PASSWORD = "SYSDBA"; + private static final int DOCKET_PORT = 5236; + private static final int JDBC_PORT = 5336; + private static final String DM_URL = "jdbc:dm://" + HOST + ":%s"; + + private static final String DRIVER_CLASS = "dm.jdbc.driver.DmDriver"; + + private static final List CONFIG_FILE = + Lists.newArrayList("/jdbc_dm_source_and_dm_upset_sink.conf"); + private static final String CREATE_SQL = + "create table if not exists %s" + + "(\n" + + " DM_BIT BIT,\n" + + " DM_INT INT,\n" + + " DM_INTEGER INTEGER,\n" + + " DM_TINYINT TINYINT,\n" + + "\n" + + " DM_BYTE BYTE,\n" + + " DM_SMALLINT SMALLINT,\n" + + " DM_BIGINT BIGINT,\n" + + "\n" + + " DM_NUMBER NUMBER,\n" + + " DM_DECIMAL DECIMAL,\n" + + " DM_FLOAT FLOAT,\n" + + " DM_DOUBLE_PRECISION DOUBLE PRECISION,\n" + + " DM_DOUBLE DOUBLE,\n" + + "\n" + + " DM_CHAR CHAR,\n" + + " DM_VARCHAR VARCHAR,\n" + + " DM_VARCHAR2 VARCHAR2,\n" + + " DM_TEXT TEXT,\n" + + " DM_LONG LONG,\n" + + "\n" + + " DM_TIMESTAMP TIMESTAMP,\n" + + " DM_DATETIME DATETIME,\n" + + " DM_DATE DATE\n" + + ")"; + private static final String CREATE_SINKTABLE_SQL = + "create table if not exists %s" + + "(\n" + + " DM_BIT BIT,\n" + + " DM_INT INT,\n" + + " DM_INTEGER INTEGER,\n" + + " DM_TINYINT TINYINT,\n" + + "\n" + + " DM_BYTE BYTE,\n" + + " DM_SMALLINT SMALLINT,\n" + + " DM_BIGINT BIGINT,\n" + + "\n" + + " DM_NUMBER NUMBER,\n" + + " DM_DECIMAL DECIMAL,\n" + + " DM_FLOAT FLOAT,\n" + + " DM_DOUBLE_PRECISION DOUBLE PRECISION,\n" + + " DM_DOUBLE DOUBLE,\n" + + "\n" + + " DM_CHAR CHAR,\n" + + " DM_VARCHAR VARCHAR,\n" + + " DM_VARCHAR2 VARCHAR2,\n" + + " DM_TEXT TEXT,\n" + + " DM_LONG LONG,\n" + + "\n" + + " DM_TIMESTAMP TIMESTAMP,\n" + + " DM_DATETIME DATETIME,\n" + + " DM_DATE DATE,\n" + + " CONSTRAINT DMPKID PRIMARY KEY (DM_BIT) \n" + + ")"; + + @Override + JdbcCase getJdbcCase() { + Map containerEnv = new HashMap<>(); + String jdbcUrl = String.format(DM_URL, JDBC_PORT); + Pair> testDataSet = initTestData(); + String[] fieldNames = testDataSet.getKey(); + + String insertSql = insertTable(DM_DATABASE, DM_SOURCE, fieldNames); + + return JdbcCase.builder() + .dockerImage(DM_IMAGE) + .networkAliases(DM_CONTAINER_HOST) + .containerEnv(containerEnv) + .driverClass(DRIVER_CLASS) + .host(HOST) + .port(DOCKET_PORT) + .localPort(DOCKET_PORT) + .jdbcTemplate(DM_URL) + .jdbcUrl(jdbcUrl) + .userName(DM_USERNAME) + .password(DM_PASSWORD) + .database(DM_DATABASE) + .sourceTable(DM_SOURCE) + .sinkTable(DM_SINK) + .createSql(CREATE_SQL) + .configFile(CONFIG_FILE) + .insertSql(insertSql) + .testData(testDataSet) + .build(); + } + + @Override + void compareResult() {} + + @Override + protected void createNeededTables() { + try (Statement statement = connection.createStatement()) { + String createTemplate = jdbcCase.getCreateSql(); + + String createSource = + String.format( + createTemplate, + buildTableInfoWithSchema( + jdbcCase.getDatabase(), jdbcCase.getSourceTable())); + String createSink = + String.format( + CREATE_SINKTABLE_SQL, + buildTableInfoWithSchema( + jdbcCase.getDatabase(), jdbcCase.getSinkTable())); + + statement.execute(createSource); + statement.execute(createSink); + connection.commit(); + } catch (Exception exception) { + throw new SeaTunnelRuntimeException(JdbcITErrorCode.CREATE_TABLE_FAILED, exception); + } + } + + @Override + String driverUrl() { + return "https://repo1.maven.org/maven2/com/dameng/DmJdbcDriver18/8.1.1.193/DmJdbcDriver18-8.1.1.193.jar"; + } + + @Override + Pair> initTestData() { + String[] fieldNames = + new String[] { + "DM_BIT", + "DM_INT", + "DM_INTEGER", + "DM_TINYINT", + "DM_BYTE", + "DM_SMALLINT", + "DM_BIGINT", + "DM_NUMBER", + "DM_DECIMAL", + "DM_FLOAT", + "DM_DOUBLE_PRECISION", + "DM_DOUBLE", + "DM_CHAR", + "DM_VARCHAR", + "DM_VARCHAR2", + "DM_TEXT", + "DM_LONG", + "DM_TIMESTAMP", + "DM_DATETIME", + "DM_DATE" + }; + + List rows = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + SeaTunnelRow row = + new SeaTunnelRow( + new Object[] { + i % 2 == 0 ? (byte) 1 : (byte) 0, + i, + i, + Short.valueOf("1"), + Byte.valueOf("1"), + i, + Long.parseLong("1"), + BigDecimal.valueOf(i, 18), + BigDecimal.valueOf(i, 18), + Float.parseFloat("1.1"), + Double.parseDouble("1.1"), + Double.parseDouble("1.1"), + 'f', + String.format("f1_%s", i), + String.format("f1_%s", i), + String.format("f1_%s", i), + String.format("{\"aa\":\"bb_%s\"}", i), + Timestamp.valueOf(LocalDateTime.now()), + new Timestamp(System.currentTimeMillis()), + Date.valueOf(LocalDate.now()) + }); + rows.add(row); + } + + return Pair.of(fieldNames, rows); + } + + @Override + protected GenericContainer initContainer() { + GenericContainer container = + new GenericContainer<>(DM_IMAGE) + .withNetwork(NETWORK) + .withNetworkAliases(DM_CONTAINER_HOST) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(DM_IMAGE))); + container.setPortBindings( + Lists.newArrayList(String.format("%s:%s", JDBC_PORT, DOCKET_PORT))); + + return container; + } + + @Override + public String quoteIdentifier(String field) { + return "\"" + field + "\""; + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/resources/jdbc_dm_source_and_dm_upset_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/resources/jdbc_dm_source_and_dm_upset_sink.conf new file mode 100644 index 000000000000..96046c88f829 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-2/src/test/resources/jdbc_dm_source_and_dm_upset_sink.conf @@ -0,0 +1,49 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + Jdbc { + url = "jdbc:dm://e2e_dmdb_upset:5236" + driver = "dm.jdbc.driver.DmDriver" + connection_check_timeout_sec = 1000 + user = "SYSDBA" + password = "SYSDBA" + query = "select * from SYSDBA.E2E_TABLE_SOURCE_UPSET" + } + +} + +sink { + Jdbc { + url = "jdbc:dm://e2e_dmdb_upset:5236" + driver = "dm.jdbc.driver.DmDriver" + connection_check_timeout_sec = 1000 + user = "SYSDBA" + password = "SYSDBA" + database = "SYSDBA" + primary_keys = [DM_BIT] + table = "E2E_TABLE_SINK_UPSET" + generate_sink_sql = true + query = "" + } +} + From 4cc10e83e774ff14e061b669aee888d20b021b3f Mon Sep 17 00:00:00 2001 From: EchoLee5 <39044001+EchoLee5@users.noreply.github.com> Date: Fri, 21 Jul 2023 14:24:26 +0800 Subject: [PATCH 09/43] [Hotfix][Connector][Jdbc] Fix the problem of JdbcOutputFormat database connection leak (#4802) [Hotfix][Connector][Jdbc] Fix the problem of JdbcOutputFormat database connection leak --- .../seatunnel/jdbc/internal/JdbcOutputFormat.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/JdbcOutputFormat.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/JdbcOutputFormat.java index ee3c4a7a68bc..d47814f15314 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/JdbcOutputFormat.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/JdbcOutputFormat.java @@ -219,10 +219,11 @@ public synchronized void close() { flush(); } catch (Exception e) { LOG.warn("Writing records to JDBC failed.", e); - throw new JdbcConnectorException( - CommonErrorCode.FLUSH_DATA_FAILED, - "Writing records to JDBC failed.", - e); + flushException = + new JdbcConnectorException( + CommonErrorCode.FLUSH_DATA_FAILED, + "Writing records to JDBC failed.", + e); } } From 6f30b296627c9213026e8d502d9052646910e542 Mon Sep 17 00:00:00 2001 From: monster <60029759+MonsterChenzhuo@users.noreply.github.com> Date: Fri, 21 Jul 2023 15:00:13 +0800 Subject: [PATCH 10/43] [Hotfix]Fix mongodb cdc e2e instability (#5128) Co-authored-by: chenzy15 --- .../cdc/mongodb/config/MongodbSourceOptions.java | 5 +++-- .../src/test/java/mongodb/MongodbCDCIT.java | 12 ++++++++++-- .../src/test/resources/mongodbcdc_to_mysql.conf | 6 +----- 3 files changed, 14 insertions(+), 9 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java index df73772e0718..dac939777f7f 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.configuration.SingleChoiceOption; import org.apache.seatunnel.connectors.cdc.base.option.SourceOptions; import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; import org.apache.seatunnel.connectors.cdc.base.option.StopMode; @@ -234,7 +235,7 @@ public class MongodbSourceOptions extends SourceOptions { .withDescription( "Decides if the table options contains Debezium client properties that start with prefix 'debezium'."); - public static final Option STARTUP_MODE = + public static final SingleChoiceOption STARTUP_MODE = Options.key(SourceOptions.STARTUP_MODE_KEY) .singleChoice( StartupMode.class, @@ -245,7 +246,7 @@ public class MongodbSourceOptions extends SourceOptions { "Optional startup mode for CDC source, valid enumerations are " + "\"initial\", \"earliest\", \"latest\", \"timestamp\"\n or \"specific\""); - public static final Option STOP_MODE = + public static final SingleChoiceOption STOP_MODE = Options.key(SourceOptions.STOP_MODE_KEY) .singleChoice(StopMode.class, Collections.singletonList(StopMode.NEVER)) .defaultValue(StopMode.NEVER) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java index dd7f985f1760..c01b36ef188c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java @@ -34,7 +34,9 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; import org.testcontainers.containers.Container; +import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerLoggerFactory; import com.mongodb.client.MongoClient; import com.mongodb.client.MongoClients; @@ -78,7 +80,7 @@ public class MongodbCDCIT extends TestSuiteBase implements TestResource { // ---------------------------------------------------------------------------- // mysql - private static final String MYSQL_HOST = "mysql_cdc_e2e"; + private static final String MYSQL_HOST = "mysql_e2e"; private static final String MYSQL_USER_NAME = "st_user"; @@ -104,8 +106,10 @@ private static MySqlContainer createMySqlContainer() { mySqlContainer.withDatabaseName(MYSQL_DATABASE); mySqlContainer.withUsername(MYSQL_USER_NAME); mySqlContainer.withPassword(MYSQL_USER_PASSWORD); + mySqlContainer.withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger("Mysql-Docker-Image"))); // For local test use - // mySqlContainer.setPortBindings(Collections.singletonList("3308:3306")); + mySqlContainer.setPortBindings(Collections.singletonList("3310:3306")); return mySqlContainer; } @@ -134,6 +138,9 @@ public void startUp() { mongodbContainer = new MongoDBContainer(NETWORK); // For local test use mongodbContainer.setPortBindings(Collections.singletonList("27017:27017")); + mongodbContainer.withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger("Mongodb-Docker-Image"))); + Startables.deepStart(Stream.of(mongodbContainer)).join(); mongodbContainer.executeCommandFileInSeparateDatabase(MONGODB_DATABASE); initConnection(); @@ -213,6 +220,7 @@ private List> querySql() { for (int i = 1; i <= columnCount; i++) { objects.add(resultSet.getObject(i)); } + log.info("Print mysql sink data:" + objects); result.add(objects); } return result; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf index 7e4a492390ba..12846c6a0c21 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf @@ -14,9 +14,6 @@ # See the License for the specific language governing permissions and # limitations under the License. # -###### -###### This config file is a demonstration of streaming processing in seatunnel config -###### env { # You can set engine configuration here @@ -45,11 +42,10 @@ source { sink { jdbc { - url = "jdbc:mysql://mysql_cdc_e2e:3306?useSSL=false&useUnicode=true&characterEncoding=UTF-8&allowPublicKeyRetrieval=false&useJDBCCompliantTimezoneShift=true&useLegacyDatetimeCode=false&serverTimezone=GMT%2B8" + url = "jdbc:mysql://mysql_e2e:3306/mongodb_cdc" driver = "com.mysql.cj.jdbc.Driver" user = "st_user" password = "seatunnel" - generate_sink_sql = true # You need to configure both database and table database = mongodb_cdc From a6c7ca138b63a7da7bff40f4840703952cb8b41f Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 24 Jul 2023 10:51:08 +0800 Subject: [PATCH 11/43] [Hotfix][Zeta] Fix task state memory leak (#5139) --- .../org/apache/seatunnel/engine/server/task/SeaTunnelTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java index 9666bdb35880..c752d45d6997 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/SeaTunnelTask.java @@ -341,7 +341,7 @@ public void ack(Barrier barrier) { new TaskAcknowledgeOperation( this.taskLocation, (CheckpointBarrier) barrier, - checkpointStates.get(barrier.getId()))) + checkpointStates.remove(barrier.getId()))) .join(); } } From 9dab2feb65661df107a8617c156c3ac078212d53 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 24 Jul 2023 10:51:40 +0800 Subject: [PATCH 12/43] [Hotfix][Zeta] Fix checkpoint error report without msg (#5137) --- .../CheckpointErrorReportOperation.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java index 75ec924b70ae..967e1572311c 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/operation/CheckpointErrorReportOperation.java @@ -23,8 +23,12 @@ import org.apache.seatunnel.engine.server.serializable.CheckpointDataSerializerHook; import org.apache.seatunnel.engine.server.task.operation.TaskOperation; +import com.hazelcast.nio.ObjectDataInput; +import com.hazelcast.nio.ObjectDataOutput; import lombok.NoArgsConstructor; +import java.io.IOException; + @NoArgsConstructor public class CheckpointErrorReportOperation extends TaskOperation { @@ -44,6 +48,18 @@ public void run() throws Exception { .reportCheckpointErrorFromTask(taskLocation, errorMsg); } + @Override + protected void writeInternal(ObjectDataOutput out) throws IOException { + super.writeInternal(out); + out.writeString(errorMsg); + } + + @Override + protected void readInternal(ObjectDataInput in) throws IOException { + super.readInternal(in); + errorMsg = in.readString(); + } + @Override public int getFactoryId() { return CheckpointDataSerializerHook.FACTORY_ID; From 75198176306fd7184d308da01a0ee33a3ef70c64 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 24 Jul 2023 10:52:22 +0800 Subject: [PATCH 13/43] [Improve][Zeta] Improve CheckpointCoordinator notify complete when restore (#5136) * [Improve] Improve CheckpointCoordinator notify complete when restore * update --- .../checkpoint/CheckpointCloseReason.java | 3 ++- .../checkpoint/CheckpointCoordinator.java | 25 ++++++++++++++++--- .../server/checkpoint/CheckpointManager.java | 4 +-- .../engine/server/master/JobMaster.java | 5 +++- .../engine/server/master/JobMasterTest.java | 2 +- 5 files changed, 30 insertions(+), 9 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCloseReason.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCloseReason.java index ae1af4d41fa1..9f35f62fd608 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCloseReason.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCloseReason.java @@ -26,7 +26,8 @@ public enum CheckpointCloseReason { CHECKPOINT_COORDINATOR_RESET("CheckpointCoordinator reset."), CHECKPOINT_INSIDE_ERROR("CheckpointCoordinator inside have error."), AGGREGATE_COMMIT_ERROR("Aggregate commit error."), - TASK_NOT_ALL_READY_WHEN_SAVEPOINT("Task not all ready, savepoint error"); + TASK_NOT_ALL_READY_WHEN_SAVEPOINT("Task not all ready, savepoint error"), + CHECKPOINT_NOTIFY_COMPLETE_FAILED("Checkpoint notify complete failed"); private final String message; diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java index 9e0ef2a53ab1..9bf0e7706967 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java @@ -272,7 +272,8 @@ private void handleCoordinatorError(CheckpointCloseReason reason, Throwable e) { checkpointCoordinatorFuture.complete( new CheckpointCoordinatorState( CheckpointCoordinatorStatus.FAILED, errorByPhysicalVertex.get())); - checkpointManager.handleCheckpointError(pipelineId); + checkpointManager.handleCheckpointError( + pipelineId, reason.equals(CheckpointCloseReason.CHECKPOINT_NOTIFY_COMPLETE_FAILED)); } private void restoreTaskState(TaskLocation taskLocation) { @@ -316,9 +317,26 @@ private void allTaskReady() { isAllTaskReady = true; InvocationFuture[] futures = notifyTaskStart(); CompletableFuture.allOf(futures).join(); + notifyCompleted(latestCompletedCheckpoint); scheduleTriggerPendingCheckpoint(coordinatorConfig.getCheckpointInterval()); } + private void notifyCompleted(CompletedCheckpoint completedCheckpoint) { + if (completedCheckpoint != null) { + try { + LOG.info("start notify checkpoint completed, checkpoint:{}", completedCheckpoint); + InvocationFuture[] invocationFutures = + notifyCheckpointCompleted(completedCheckpoint.getCheckpointId()); + CompletableFuture.allOf(invocationFutures).join(); + } catch (Throwable e) { + handleCoordinatorError( + "notify checkpoint completed failed", + e, + CheckpointCloseReason.CHECKPOINT_NOTIFY_COMPLETE_FAILED); + } + } + } + public InvocationFuture[] notifyTaskStart() { return plan.getPipelineSubtasks().stream() .map(NotifyTaskStartOperation::new) @@ -358,6 +376,7 @@ protected void restoreCoordinator(boolean alreadyStarted) { shutdown = false; if (alreadyStarted) { isAllTaskReady = true; + notifyCompleted(latestCompletedCheckpoint); tryTriggerPendingCheckpoint(CHECKPOINT_TYPE); } else { isAllTaskReady = false; @@ -719,10 +738,8 @@ public synchronized void completePendingCheckpoint(CompletedCheckpoint completed completedCheckpoint.getCheckpointId(), completedCheckpoint.getPipelineId(), completedCheckpoint.getJobId()); - InvocationFuture[] invocationFutures = notifyCheckpointCompleted(checkpointId); - CompletableFuture.allOf(invocationFutures).join(); - // TODO: notifyCheckpointCompleted fail latestCompletedCheckpoint = completedCheckpoint; + notifyCompleted(completedCheckpoint); if (isCompleted()) { cleanPendingCheckpoint(CheckpointCloseReason.CHECKPOINT_COORDINATOR_COMPLETED); if (latestCompletedCheckpoint.getCheckpointType().equals(SAVEPOINT_TYPE)) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java index f34ae2f6a0ad..0c5a91698e7b 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointManager.java @@ -174,8 +174,8 @@ public void reportedPipelineRunning(int pipelineId, boolean alreadyStarted) { getCheckpointCoordinator(pipelineId).restoreCoordinator(alreadyStarted); } - protected void handleCheckpointError(int pipelineId) { - jobMaster.handleCheckpointError(pipelineId); + protected void handleCheckpointError(int pipelineId, boolean neverRestore) { + jobMaster.handleCheckpointError(pipelineId, neverRestore); } private CheckpointCoordinator getCheckpointCoordinator(TaskLocation taskLocation) { diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java index 13b89a69dd94..11cc5f21b0b5 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java @@ -325,7 +325,10 @@ public void run() { } } - public void handleCheckpointError(long pipelineId) { + public void handleCheckpointError(long pipelineId, boolean neverRestore) { + if (neverRestore) { + this.neverNeedRestore(); + } this.physicalPlan .getPipelineList() .forEach( diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/master/JobMasterTest.java b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/master/JobMasterTest.java index efdc9e0f71f4..bb331bd0181e 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/master/JobMasterTest.java +++ b/seatunnel-engine/seatunnel-engine-server/src/test/java/org/apache/seatunnel/engine/server/master/JobMasterTest.java @@ -139,7 +139,7 @@ public void testHandleCheckpointTimeout() throws Exception { jobMaster.neverNeedRestore(); // call checkpoint timeout - jobMaster.handleCheckpointError(1); + jobMaster.handleCheckpointError(1, false); PassiveCompletableFuture jobMasterCompleteFuture = jobMaster.getJobMasterCompleteFuture(); From 2391135388c612b0bc0a6ae45e08afa9ceb0822d Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 24 Jul 2023 10:52:50 +0800 Subject: [PATCH 14/43] [Improve][Zeta] Improve CheckpointCoordinator log error when report error from task (#178) (#5134) --- .../engine/server/checkpoint/CheckpointCoordinator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java index 9bf0e7706967..0f9e03df349c 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java @@ -346,7 +346,9 @@ public InvocationFuture[] notifyTaskStart() { public void reportCheckpointErrorFromTask(String errorMsg) { handleCoordinatorError( - CheckpointCloseReason.CHECKPOINT_INSIDE_ERROR, new SeaTunnelException(errorMsg)); + "report error from task", + new SeaTunnelException(errorMsg), + CheckpointCloseReason.CHECKPOINT_INSIDE_ERROR); } private void scheduleTriggerPendingCheckpoint(long delayMills) { From 973d0450abfed571a9ec3ae81f91c8edebd93d25 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 24 Jul 2023 10:54:13 +0800 Subject: [PATCH 15/43] [Hotfix][Zeta] Fix MultipleTableJobConfigParser ignore env option (#5067) * [Fix][Zeta] Fix MultipleTableJobConfigParser ignore env option * update --- .../core/parse/MultipleTableJobConfigParser.java | 11 +++++------ .../seatunnel/engine/server/master/JobMaster.java | 3 ++- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java index faf178e1bdd1..09027a2a248f 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java @@ -231,12 +231,11 @@ private void fillJobConfig() { jobConfig.setName(envOptions.get(EnvCommonOptions.JOB_NAME)); } envOptions - .getOptional(EnvCommonOptions.CHECKPOINT_INTERVAL) - .ifPresent( - interval -> - jobConfig - .getEnvOptions() - .put(EnvCommonOptions.CHECKPOINT_INTERVAL.key(), interval)); + .toMap() + .forEach( + (k, v) -> { + jobConfig.getEnvOptions().put(k, v); + }); } private static boolean isFallback( diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java index 11cc5f21b0b5..e14d946c8117 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobMaster.java @@ -273,7 +273,8 @@ private CheckpointConfig createJobCheckpointConfig( if (jobEnv.containsKey(EnvCommonOptions.CHECKPOINT_INTERVAL.key())) { jobCheckpointConfig.setCheckpointInterval( - (Long) jobEnv.get(EnvCommonOptions.CHECKPOINT_INTERVAL.key())); + Long.parseLong( + jobEnv.get(EnvCommonOptions.CHECKPOINT_INTERVAL.key()).toString())); } return jobCheckpointConfig; } From 1e18a8c5303fb071d6c4d5d8dc893b8882c3bc4a Mon Sep 17 00:00:00 2001 From: hailin0 Date: Mon, 24 Jul 2023 15:35:27 +0800 Subject: [PATCH 16/43] [Improve][Connector[File] Optimize files commit order (#5045) Before using `HashMap` store files path, so every checkpoint file commit is out of order. Now switch to using `LinkedHashMap` to ensure that files are commit in the generated order --- .../file/sink/BaseFileSinkWriter.java | 6 +- .../sink/commit/FileAggregatedCommitInfo.java | 6 +- .../file/sink/commit/FileCommitInfo.java | 6 +- .../commit/FileSinkAggregatedCommitter.java | 15 ++-- .../file/sink/commit/FileSinkCommitter.java | 75 ------------------- .../file/sink/state/FileSinkState.java | 6 +- .../sink/writer/AbstractWriteStrategy.java | 44 +++++++---- .../file/sink/writer/ExcelWriteStrategy.java | 7 +- .../file/sink/writer/JsonWriteStrategy.java | 5 +- .../file/sink/writer/OrcWriteStrategy.java | 6 +- .../sink/writer/ParquetWriteStrategy.java | 7 +- .../file/sink/writer/TextWriteStrategy.java | 5 +- .../file/sink/writer/WriteStrategy.java | 4 +- .../S3RedshiftSinkAggregatedCommitter.java | 5 +- 14 files changed, 68 insertions(+), 129 deletions(-) delete mode 100644 seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileSinkCommitter.java diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/BaseFileSinkWriter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/BaseFileSinkWriter.java index 7102e954a463..22200249f63d 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/BaseFileSinkWriter.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/BaseFileSinkWriter.java @@ -34,14 +34,14 @@ import java.io.IOException; import java.util.Collections; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Optional; import java.util.UUID; import java.util.stream.Collectors; public class BaseFileSinkWriter implements SinkWriter { - private final WriteStrategy writeStrategy; + protected final WriteStrategy writeStrategy; private final FileSystemUtils fileSystemUtils; @SuppressWarnings("checkstyle:MagicNumber") @@ -67,7 +67,7 @@ public BaseFileSinkWriter( List transactions = findTransactionList(jobId, uuidPrefix); FileSinkAggregatedCommitter fileSinkAggregatedCommitter = new FileSinkAggregatedCommitter(fileSystemUtils); - HashMap fileStatesMap = new HashMap<>(); + LinkedHashMap fileStatesMap = new LinkedHashMap<>(); fileSinkStates.forEach( fileSinkState -> fileStatesMap.put(fileSinkState.getTransactionId(), fileSinkState)); diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileAggregatedCommitInfo.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileAggregatedCommitInfo.java index 16d94a1f63a7..5ca3b30fade4 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileAggregatedCommitInfo.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileAggregatedCommitInfo.java @@ -21,8 +21,8 @@ import lombok.Data; import java.io.Serializable; +import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; @Data @AllArgsConstructor @@ -34,7 +34,7 @@ public class FileAggregatedCommitInfo implements Serializable { * *

V is the target file path of the data file. */ - private final Map> transactionMap; + private final LinkedHashMap> transactionMap; /** * Storage the partition information in map. @@ -43,5 +43,5 @@ public class FileAggregatedCommitInfo implements Serializable { * *

V is the list of partition column's values. */ - private final Map> partitionDirAndValuesMap; + private final LinkedHashMap> partitionDirAndValuesMap; } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileCommitInfo.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileCommitInfo.java index 86c433b8f55d..27e74ff0a87e 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileCommitInfo.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileCommitInfo.java @@ -21,8 +21,8 @@ import lombok.Data; import java.io.Serializable; +import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; @Data @AllArgsConstructor @@ -34,7 +34,7 @@ public class FileCommitInfo implements Serializable { * *

V is the target file path of the data file. */ - private final Map needMoveFiles; + private final LinkedHashMap needMoveFiles; /** * Storage the partition information in map. @@ -43,7 +43,7 @@ public class FileCommitInfo implements Serializable { * *

V is the list of partition column's values. */ - private final Map> partitionDirAndValuesMap; + private final LinkedHashMap> partitionDirAndValuesMap; /** Storage the transaction directory */ private final String transactionDir; diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileSinkAggregatedCommitter.java index b12ef1165a22..a076188e2a2c 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileSinkAggregatedCommitter.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileSinkAggregatedCommitter.java @@ -24,7 +24,7 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -44,7 +44,7 @@ public List commit( aggregatedCommitInfos.forEach( aggregatedCommitInfo -> { try { - for (Map.Entry> entry : + for (Map.Entry> entry : aggregatedCommitInfo.getTransactionMap().entrySet()) { for (Map.Entry mvFileEntry : entry.getValue().entrySet()) { @@ -77,13 +77,14 @@ public FileAggregatedCommitInfo combine(List commitInfos) { if (commitInfos == null || commitInfos.size() == 0) { return null; } - Map> aggregateCommitInfo = new HashMap<>(); - Map> partitionDirAndValuesMap = new HashMap<>(); + LinkedHashMap> aggregateCommitInfo = + new LinkedHashMap<>(); + LinkedHashMap> partitionDirAndValuesMap = new LinkedHashMap<>(); commitInfos.forEach( commitInfo -> { - Map needMoveFileMap = + LinkedHashMap needMoveFileMap = aggregateCommitInfo.computeIfAbsent( - commitInfo.getTransactionDir(), k -> new HashMap<>()); + commitInfo.getTransactionDir(), k -> new LinkedHashMap<>()); needMoveFileMap.putAll(commitInfo.getNeedMoveFiles()); if (commitInfo.getPartitionDirAndValuesMap() != null && !commitInfo.getPartitionDirAndValuesMap().isEmpty()) { @@ -109,7 +110,7 @@ public void abort(List aggregatedCommitInfos) throws E aggregatedCommitInfos.forEach( aggregatedCommitInfo -> { try { - for (Map.Entry> entry : + for (Map.Entry> entry : aggregatedCommitInfo.getTransactionMap().entrySet()) { // rollback the file for (Map.Entry mvFileEntry : diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileSinkCommitter.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileSinkCommitter.java deleted file mode 100644 index 6525b5e7d4b6..000000000000 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/commit/FileSinkCommitter.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.file.sink.commit; - -import org.apache.seatunnel.api.sink.SinkCommitter; -import org.apache.seatunnel.connectors.seatunnel.file.sink.util.FileSystemUtils; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -/** Deprecated interface since 2.3.0-beta, now used {@link FileSinkAggregatedCommitter} */ -@Deprecated -public class FileSinkCommitter implements SinkCommitter { - private final FileSystemUtils fileSystemUtils; - - public FileSinkCommitter(FileSystemUtils fileSystemUtils) { - this.fileSystemUtils = fileSystemUtils; - } - - @Override - public List commit(List commitInfos) throws IOException { - ArrayList failedCommitInfos = new ArrayList<>(); - for (FileCommitInfo commitInfo : commitInfos) { - Map needMoveFiles = commitInfo.getNeedMoveFiles(); - needMoveFiles.forEach( - (k, v) -> { - try { - fileSystemUtils.renameFile(k, v, true); - } catch (IOException e) { - failedCommitInfos.add(commitInfo); - } - }); - fileSystemUtils.deleteFile(commitInfo.getTransactionDir()); - } - return failedCommitInfos; - } - - /** - * Abort the transaction, this method will be called (**Only** on Spark engine) when the commit - * is failed. - * - * @param commitInfos The list of commit message, used to abort the commit. - * @throws IOException throw IOException when close failed. - */ - @Override - public void abort(List commitInfos) throws IOException { - for (FileCommitInfo commitInfo : commitInfos) { - Map needMoveFiles = commitInfo.getNeedMoveFiles(); - for (Map.Entry entry : needMoveFiles.entrySet()) { - if (fileSystemUtils.fileExist(entry.getValue()) - && !fileSystemUtils.fileExist(entry.getKey())) { - fileSystemUtils.renameFile(entry.getValue(), entry.getKey(), true); - } - } - fileSystemUtils.deleteFile(commitInfo.getTransactionDir()); - } - } -} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/state/FileSinkState.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/state/FileSinkState.java index 7d28df230514..34ca13625f60 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/state/FileSinkState.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/state/FileSinkState.java @@ -21,8 +21,8 @@ import lombok.Data; import java.io.Serializable; +import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; @Data @AllArgsConstructor @@ -30,7 +30,7 @@ public class FileSinkState implements Serializable { private final String transactionId; private final String uuidPrefix; private final Long checkpointId; - private final Map needMoveFiles; - private final Map> partitionDirAndValuesMap; + private final LinkedHashMap needMoveFiles; + private final LinkedHashMap> partitionDirAndValuesMap; private final String transactionDir; } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/AbstractWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/AbstractWriteStrategy.java index 6820d28d8558..f3160eec7e60 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/AbstractWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/AbstractWriteStrategy.java @@ -50,6 +50,7 @@ import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -76,9 +77,9 @@ public abstract class AbstractWriteStrategy implements WriteStrategy { protected String uuidPrefix; protected String transactionDirectory; - protected Map needMoveFiles; - protected Map beingWrittenFile = new HashMap<>(); - private Map> partitionDirAndValuesMap; + protected LinkedHashMap needMoveFiles; + protected LinkedHashMap beingWrittenFile = new LinkedHashMap<>(); + private LinkedHashMap> partitionDirAndValuesMap; protected SeaTunnelRowType seaTunnelRowType; // Checkpoint id from engine is start with 1 @@ -111,13 +112,18 @@ public void init(HadoopConf conf, String jobId, String uuidPrefix, int subTaskIn @Override public void write(SeaTunnelRow seaTunnelRow) throws FileConnectorException { if (currentBatchSize >= batchSize) { - this.partId++; + newFilePart(); currentBatchSize = 0; - beingWrittenFile.clear(); } currentBatchSize++; } + public synchronized void newFilePart() { + this.partId++; + beingWrittenFile.clear(); + log.debug("new file part: {}", partId); + } + protected SeaTunnelRowType buildSchemaWithRowType( SeaTunnelRowType seaTunnelRowType, List sinkColumnsIndex) { SeaTunnelDataType[] fieldTypes = seaTunnelRowType.getFieldTypes(); @@ -177,9 +183,9 @@ public void setSeaTunnelRowTypeInfo(SeaTunnelRowType seaTunnelRowType) { * @return the map of partition directory */ @Override - public Map> generatorPartitionDir(SeaTunnelRow seaTunnelRow) { + public LinkedHashMap> generatorPartitionDir(SeaTunnelRow seaTunnelRow) { List partitionFieldsIndexInRow = fileSinkConfig.getPartitionFieldsIndexInRow(); - Map> partitionDirAndValuesMap = new HashMap<>(1); + LinkedHashMap> partitionDirAndValuesMap = new LinkedHashMap<>(1); if (CollectionUtils.isEmpty(partitionFieldsIndexInRow)) { partitionDirAndValuesMap.put(BaseSinkConfig.NON_PARTITION, null); return partitionDirAndValuesMap; @@ -258,12 +264,15 @@ public String generateFileName(String transactionId) { @Override public Optional prepareCommit() { this.finishAndCloseFile(); - Map commitMap = new HashMap<>(this.needMoveFiles); - Map> copyMap = + LinkedHashMap commitMap = new LinkedHashMap<>(this.needMoveFiles); + LinkedHashMap> copyMap = this.partitionDirAndValuesMap.entrySet().stream() .collect( Collectors.toMap( - Map.Entry::getKey, e -> new ArrayList<>(e.getValue()))); + Map.Entry::getKey, + e -> new ArrayList<>(e.getValue()), + (e1, e2) -> e1, + LinkedHashMap::new)); return Optional.of(new FileCommitInfo(commitMap, copyMap, transactionDirectory)); } @@ -301,8 +310,8 @@ public void beginTransaction(Long checkpointId) { this.checkpointId = checkpointId; this.transactionId = getTransactionId(checkpointId); this.transactionDirectory = getTransactionDir(this.transactionId); - this.needMoveFiles = new HashMap<>(); - this.partitionDirAndValuesMap = new HashMap<>(); + this.needMoveFiles = new LinkedHashMap<>(); + this.partitionDirAndValuesMap = new LinkedHashMap<>(); } private String getTransactionId(Long checkpointId) { @@ -325,18 +334,21 @@ private String getTransactionId(Long checkpointId) { */ @Override public List snapshotState(long checkpointId) { - Map> commitMap = + LinkedHashMap> commitMap = this.partitionDirAndValuesMap.entrySet().stream() .collect( Collectors.toMap( - Map.Entry::getKey, e -> new ArrayList<>(e.getValue()))); + Map.Entry::getKey, + e -> new ArrayList<>(e.getValue()), + (e1, e2) -> e1, + LinkedHashMap::new)); ArrayList fileState = Lists.newArrayList( new FileSinkState( this.transactionId, this.uuidPrefix, this.checkpointId, - new HashMap<>(this.needMoveFiles), + new LinkedHashMap<>(this.needMoveFiles), commitMap, this.getTransactionDir(transactionId))); this.beingWrittenFile.clear(); @@ -363,7 +375,7 @@ public static String getTransactionDirPrefix(String tmpPath, String jobId, Strin } public String getOrCreateFilePathBeingWritten(@NonNull SeaTunnelRow seaTunnelRow) { - Map> dataPartitionDirAndValuesMap = + LinkedHashMap> dataPartitionDirAndValuesMap = generatorPartitionDir(seaTunnelRow); String beingWrittenFileKey = dataPartitionDirAndValuesMap.keySet().toArray()[0].toString(); // get filePath from beingWrittenFile diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/ExcelWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/ExcelWriteStrategy.java index bb8d09d30fde..d5786ea2f8c6 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/ExcelWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/ExcelWriteStrategy.java @@ -28,15 +28,14 @@ import lombok.NonNull; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; +import java.util.LinkedHashMap; public class ExcelWriteStrategy extends AbstractWriteStrategy { - private final Map beingWrittenWriter; + private final LinkedHashMap beingWrittenWriter; public ExcelWriteStrategy(FileSinkConfig fileSinkConfig) { super(fileSinkConfig); - this.beingWrittenWriter = new HashMap<>(); + this.beingWrittenWriter = new LinkedHashMap<>(); } @Override diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/JsonWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/JsonWriteStrategy.java index c16f6135770c..c72a38068df3 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/JsonWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/JsonWriteStrategy.java @@ -33,17 +33,18 @@ import java.io.IOException; import java.io.OutputStream; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; public class JsonWriteStrategy extends AbstractWriteStrategy { private final byte[] rowDelimiter; private SerializationSchema serializationSchema; - private final Map beingWrittenOutputStream; + private final LinkedHashMap beingWrittenOutputStream; private final Map isFirstWrite; public JsonWriteStrategy(FileSinkConfig textFileSinkConfig) { super(textFileSinkConfig); - this.beingWrittenOutputStream = new HashMap<>(); + this.beingWrittenOutputStream = new LinkedHashMap<>(); this.isFirstWrite = new HashMap<>(); this.rowDelimiter = textFileSinkConfig.getRowDelimiter().getBytes(); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/OrcWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/OrcWriteStrategy.java index 551d02f5b97a..0e55b46e26f0 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/OrcWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/OrcWriteStrategy.java @@ -55,16 +55,16 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.time.temporal.ChronoField; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; public class OrcWriteStrategy extends AbstractWriteStrategy { - private final Map beingWrittenWriter; + private final LinkedHashMap beingWrittenWriter; public OrcWriteStrategy(FileSinkConfig fileSinkConfig) { super(fileSinkConfig); - this.beingWrittenWriter = new HashMap<>(); + this.beingWrittenWriter = new LinkedHashMap<>(); } @Override diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/ParquetWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/ParquetWriteStrategy.java index ce104da80084..8c2c93820080 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/ParquetWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/ParquetWriteStrategy.java @@ -57,15 +57,14 @@ import java.time.LocalDateTime; import java.time.ZoneOffset; import java.util.ArrayList; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import java.util.stream.IntStream; @SuppressWarnings("checkstyle:MagicNumber") public class ParquetWriteStrategy extends AbstractWriteStrategy { - private final Map> beingWrittenWriter; + private final LinkedHashMap> beingWrittenWriter; private AvroSchemaConverter schemaConverter; private Schema schema; public static final int[] PRECISION_TO_BYTE_COUNT = new int[38]; @@ -80,7 +79,7 @@ public class ParquetWriteStrategy extends AbstractWriteStrategy { public ParquetWriteStrategy(FileSinkConfig fileSinkConfig) { super(fileSinkConfig); - this.beingWrittenWriter = new HashMap<>(); + this.beingWrittenWriter = new LinkedHashMap<>(); } @Override diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/TextWriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/TextWriteStrategy.java index 7e94e13c9615..f309edb70f26 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/TextWriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/TextWriteStrategy.java @@ -36,10 +36,11 @@ import java.io.IOException; import java.io.OutputStream; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; public class TextWriteStrategy extends AbstractWriteStrategy { - private final Map beingWrittenOutputStream; + private final LinkedHashMap beingWrittenOutputStream; private final Map isFirstWrite; private final String fieldDelimiter; private final String rowDelimiter; @@ -50,7 +51,7 @@ public class TextWriteStrategy extends AbstractWriteStrategy { public TextWriteStrategy(FileSinkConfig fileSinkConfig) { super(fileSinkConfig); - this.beingWrittenOutputStream = new HashMap<>(); + this.beingWrittenOutputStream = new LinkedHashMap<>(); this.isFirstWrite = new HashMap<>(); this.fieldDelimiter = fileSinkConfig.getFieldDelimiter(); this.rowDelimiter = fileSinkConfig.getRowDelimiter(); diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/WriteStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/WriteStrategy.java index 6d75de29c6c8..a64af87d061d 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/WriteStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sink/writer/WriteStrategy.java @@ -27,8 +27,8 @@ import org.apache.hadoop.conf.Configuration; import java.io.Serializable; +import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; public interface WriteStrategy extends Transaction, Serializable { /** @@ -67,7 +67,7 @@ public interface WriteStrategy extends Transaction, Serializable { * @param seaTunnelRow seaTunnelRow * @return the map of partition directory */ - Map> generatorPartitionDir(SeaTunnelRow seaTunnelRow); + LinkedHashMap> generatorPartitionDir(SeaTunnelRow seaTunnelRow); /** * use transaction id generate file name diff --git a/seatunnel-connectors-v2/connector-s3-redshift/src/main/java/org/apache/seatunnel/connectors/seatunnel/redshift/commit/S3RedshiftSinkAggregatedCommitter.java b/seatunnel-connectors-v2/connector-s3-redshift/src/main/java/org/apache/seatunnel/connectors/seatunnel/redshift/commit/S3RedshiftSinkAggregatedCommitter.java index 97476fafc568..620bea134b76 100644 --- a/seatunnel-connectors-v2/connector-s3-redshift/src/main/java/org/apache/seatunnel/connectors/seatunnel/redshift/commit/S3RedshiftSinkAggregatedCommitter.java +++ b/seatunnel-connectors-v2/connector-s3-redshift/src/main/java/org/apache/seatunnel/connectors/seatunnel/redshift/commit/S3RedshiftSinkAggregatedCommitter.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.sql.SQLException; import java.util.ArrayList; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -58,7 +59,7 @@ public List commit( aggregatedCommitInfos.forEach( aggregatedCommitInfo -> { try { - for (Map.Entry> entry : + for (Map.Entry> entry : aggregatedCommitInfo.getTransactionMap().entrySet()) { for (Map.Entry mvFileEntry : entry.getValue().entrySet()) { @@ -92,7 +93,7 @@ public void abort(List aggregatedCommitInfos) { aggregatedCommitInfos.forEach( aggregatedCommitInfo -> { try { - for (Map.Entry> entry : + for (Map.Entry> entry : aggregatedCommitInfo.getTransactionMap().entrySet()) { // delete the transaction dir fileSystemUtils.deleteFile(entry.getKey()); From e964c03dca69f3a95da72fee2fba68e5cf24485d Mon Sep 17 00:00:00 2001 From: monster <60029759+MonsterChenzhuo@users.noreply.github.com> Date: Tue, 25 Jul 2023 10:09:11 +0800 Subject: [PATCH 17/43] [Hotfix][Mongodb cdc] Solve startup resume token is negative (#5143) --------- Co-authored-by: chenzy15 --- .../source/dialect/MongodbDialect.java | 7 +++ .../cdc/mongodb/utils/ResumeToken.java | 46 ++++++++----------- .../test/java/mongodb/MongoDBContainer.java | 1 + 3 files changed, 26 insertions(+), 28 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java index 11ef57ffc5f9..25e463c17e56 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java @@ -34,6 +34,7 @@ import com.mongodb.client.MongoClient; import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; import javax.annotation.Nonnull; @@ -52,6 +53,7 @@ import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getCurrentClusterTime; import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getLatestResumeToken; +@Slf4j public class MongodbDialect implements DataSourceDialect { private final Map cache = @@ -137,6 +139,11 @@ public ChangeStreamOffset displayCurrentOffset(MongodbSourceConfig sourceConfig) ChangeStreamOffset changeStreamOffset; if (startupResumeToken != null) { changeStreamOffset = new ChangeStreamOffset(startupResumeToken); + log.info( + "startup resume token={},change stream offset={}", + startupResumeToken, + changeStreamOffset); + } else { changeStreamOffset = new ChangeStreamOffset(getCurrentClusterTime(mongoClient)); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java index 3ddd2ccbb21a..5ee8962bc532 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java @@ -17,8 +17,6 @@ package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; -import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; - import org.bson.BsonDocument; import org.bson.BsonTimestamp; import org.bson.BsonValue; @@ -29,41 +27,33 @@ import java.nio.ByteOrder; import java.util.Objects; -import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; - public class ResumeToken { private static final int K_TIMESTAMP = 130; - public static @Nonnull BsonTimestamp decodeTimestamp(BsonDocument resumeToken) { - Objects.requireNonNull(resumeToken, "Missing ResumeToken."); - BsonValue bsonValue = resumeToken.get("_data"); - byte[] keyStringBytes = extractKeyStringBytes(bsonValue); - validateKeyType(keyStringBytes); - - ByteBuffer buffer = ByteBuffer.wrap(keyStringBytes).order(ByteOrder.BIG_ENDIAN); - int t = buffer.getInt(); - int i = buffer.getInt(); - return new BsonTimestamp(t, i); - } - - private static byte[] extractKeyStringBytes(@Nonnull BsonValue bsonValue) { - if (bsonValue.isBinary()) { - return bsonValue.asBinary().getData(); - } else if (bsonValue.isString()) { - return hexToUint8Array(bsonValue.asString().getValue()); + public static BsonTimestamp decodeTimestamp(BsonDocument resumeToken) { + BsonValue bsonValue = + Objects.requireNonNull(resumeToken, "Missing ResumeToken.").get("_data"); + final byte[] keyStringBytes; + // Resume Tokens format: https://www.mongodb.com/docs/manual/changeStreams/#resume-tokens + if (bsonValue.isBinary()) { // BinData + keyStringBytes = bsonValue.asBinary().getData(); + } else if (bsonValue.isString()) { // Hex-encoded string (v0 or v1) + keyStringBytes = hexToUint8Array(bsonValue.asString().getValue()); } else { - throw new MongodbConnectorException( - ILLEGAL_ARGUMENT, "Unknown resume token format: " + bsonValue); + throw new IllegalArgumentException( + "Unknown resume token format: " + resumeToken.toJson()); } - } - private static void validateKeyType(byte[] keyStringBytes) { - int kType = keyStringBytes[0] & 0xff; + ByteBuffer buffer = ByteBuffer.wrap(keyStringBytes).order(ByteOrder.BIG_ENDIAN); + int kType = buffer.get() & 0xff; if (kType != K_TIMESTAMP) { - throw new MongodbConnectorException( - ILLEGAL_ARGUMENT, "Unknown keyType of timestamp: " + kType); + throw new IllegalArgumentException("Unknown keyType of timestamp: " + kType); } + + int t = buffer.getInt(); + int i = buffer.getInt(); + return new BsonTimestamp(t, i); } private static byte[] hexToUint8Array(@Nonnull String str) { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java index c33f6d047d48..a311bccc90bd 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java @@ -84,6 +84,7 @@ public MongoDBContainer(Network network, ShardingClusterRole clusterRole) { withExposedPorts(MONGODB_PORT); withCommand(ShardingClusterRole.startupCommand(clusterRole)); waitingFor(clusterRole.waitStrategy); + withEnv("TZ", "Asia/Shanghai"); } public void executeCommand(String command) { From 53a1f0c6c14190c37c0b28320db049f822436a4f Mon Sep 17 00:00:00 2001 From: Xiaojian Sun Date: Tue, 25 Jul 2023 10:09:24 +0800 Subject: [PATCH 18/43] [Feature][connector][kafka] Support read debezium format message from kafka (#5066) --- .github/workflows/backend.yml | 2 +- docs/en/connector-v2/formats/debezium-json.md | 107 +++++ docs/en/connector-v2/sink/Kafka.md | 16 +- docs/en/connector-v2/source/kafka.md | 17 +- release-note.md | 12 +- .../seatunnel/kafka/config/Config.java | 8 +- .../seatunnel/kafka/config/MessageFormat.java | 1 + .../DefaultSeaTunnelRowSerializer.java | 3 + .../seatunnel/kafka/source/KafkaSource.java | 10 + .../kafka/source/KafkaSourceFactory.java | 1 + .../connector/kafka/DebeziumToKafkaIT.java | 418 ++++++++++++++++++ .../resources/debezium/register-mysql.json | 16 + .../kafkasource_debezium_cdc_to_pgsql.conf | 62 +++ .../kafkasource_debezium_to_kafka.conf | 57 +++ .../format/json/JsonFormatOptions.java | 13 +- .../DebeziumJsonDeserializationSchema.java | 168 +++++++ .../debezium/DebeziumJsonFormatFactory.java | 70 +++ .../debezium/DebeziumJsonFormatOptions.java | 53 +++ .../DebeziumJsonSerializationSchema.java | 80 ++++ ...apache.seatunnel.api.table.factory.Factory | 1 + .../debezium/DebeziumJsonSerDeSchemaTest.java | 163 +++++++ .../src/test/resources/debezium-data.txt | 16 + 22 files changed, 1270 insertions(+), 24 deletions(-) create mode 100644 docs/en/connector-v2/formats/debezium-json.md create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/DebeziumToKafkaIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/debezium/register-mysql.json create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_cdc_to_pgsql.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_to_kafka.conf create mode 100644 seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java create mode 100644 seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatFactory.java create mode 100644 seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java create mode 100644 seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java create mode 100644 seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java create mode 100644 seatunnel-formats/seatunnel-format-json/src/test/resources/debezium-data.txt diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml index fbe37acece53..6da4f4a5ab64 100644 --- a/.github/workflows/backend.yml +++ b/.github/workflows/backend.yml @@ -564,7 +564,7 @@ jobs: matrix: java: [ '8', '11' ] os: [ 'ubuntu-latest' ] - timeout-minutes: 90 + timeout-minutes: 150 steps: - uses: actions/checkout@v2 - name: Set up JDK ${{ matrix.java }} diff --git a/docs/en/connector-v2/formats/debezium-json.md b/docs/en/connector-v2/formats/debezium-json.md new file mode 100644 index 000000000000..4c40a0298e49 --- /dev/null +++ b/docs/en/connector-v2/formats/debezium-json.md @@ -0,0 +1,107 @@ +# Debezium Format + +Changelog-Data-Capture Format: Serialization Schema Format: Deserialization Schema + +Debezium is a set of distributed services to capture changes in your databases so that your applications can see those changes and respond to them. Debezium records all row-level changes within each database table in a *change event stream*, and applications simply read these streams to see the change events in the same order in which they occurred. + +Seatunnel supports to interpret Debezium JSON messages as INSERT/UPDATE/DELETE messages into seatunnel system. This is useful in many cases to leverage this feature, such as + + synchronizing incremental data from databases to other systems + auditing logs + real-time materialized views on databases + temporal join changing history of a database table and so on. + +Seatunnel also supports to encode the INSERT/UPDATE/DELETE messages in Seatunnel asDebezium JSON messages, and emit to storage like Kafka. + +# Format Options + +| option | default | required | Description | +|-----------------------------------|---------|----------|------------------------------------------------------------------------------------------------------| +| format | (none) | yes | Specify what format to use, here should be 'debezium_json'. | +| debezium-json.ignore-parse-errors | false | no | Skip fields and rows with parse errors instead of failing. Fields are set to null in case of errors. | + +# How to use Debezium format + +## Kafka uses example + +Debezium provides a unified format for changelog, here is a simple example for an update operation captured from a MySQL products table: + +```bash +{ + "before": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter ", + "weight": 5.18 + }, + "after": { + "id": 111, + "name": "scooter", + "description": "Big 2-wheel scooter ", + "weight": 5.17 + }, + "source": { + "version": "1.1.1.Final", + "connector": "mysql", + "name": "dbserver1", + "ts_ms": 1589362330000, + "snapshot": "false", + "db": "inventory", + "table": "products", + "server_id": 223344, + "gtid": null, + "file": "mysql-bin.000003", + "pos": 2090, + "row": 0, + "thread": 2, + "query": null + }, + "op": "u", + "ts_ms": 1589362330904, + "transaction": null +} +``` + +Note: please refer to Debezium documentation about the meaning of each fields. + +The MySQL products table has 4 columns (id, name, description and weight). +The above JSON message is an update change event on the products table where the weight value of the row with id = 111 is changed from 5.18 to 5.15. +Assuming the messages have been synchronized to Kafka topic products_binlog, then we can use the following Seatunnel conf to consume this topic and interpret the change events by Debezium format. + +```bash +env { + execution.parallelism = 1 + job.mode = "BATCH" +} + +source { + Kafka { + bootstrap.servers = "kafkaCluster:9092" + topic = "products_binlog" + result_table_name = "kafka_name" + start_mode = earliest + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "string" + } + } + format = debezium_json + } + +} + +transform { +} + +sink { + Kafka { + bootstrap.servers = "kafkaCluster:9092" + topic = "consume-binlog" + format = debezium_json + } +} +``` + diff --git a/docs/en/connector-v2/sink/Kafka.md b/docs/en/connector-v2/sink/Kafka.md index 4dbd3a84ce7f..f971e5390b0b 100644 --- a/docs/en/connector-v2/sink/Kafka.md +++ b/docs/en/connector-v2/sink/Kafka.md @@ -108,8 +108,10 @@ Kafka distinguishes different transactions by different transactionId. This para ### format -Data format. The default format is json. Optional text format. The default field separator is ",". -If you customize the delimiter, add the "field_delimiter" option. +Data format. The default format is json. Optional text format, canal-json and debezium-json. +If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option. +If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details. +If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. ### field_delimiter @@ -209,8 +211,10 @@ sink { ### next version -- [Improve] Support to specify multiple partition keys [3230](https://github.com/apache/seatunnel/pull/3230) -- [Improve] Add text format for kafka sink connector [3711](https://github.com/apache/seatunnel/pull/3711) -- [Improve] Support extract topic from SeaTunnelRow fields [3742](https://github.com/apache/seatunnel/pull/3742) -- [Improve] Change Connector Custom Config Prefix To Map [3719](https://github.com/apache/seatunnel/pull/3719) +- [Improve] Support to specify multiple partition keys [3230](https://github.com/apache/incubator-seatunnel/pull/3230) +- [Improve] Add text format for kafka sink connector [3711](https://github.com/apache/incubator-seatunnel/pull/3711) +- [Improve] Support extract topic from SeaTunnelRow fields [3742](https://github.com/apache/incubator-seatunnel/pull/3742) +- [Improve] Change Connector Custom Config Prefix To Map [3719](https://github.com/apache/incubator-seatunnel/pull/3719) +- [Improve] Support read canal format message [3950](https://github.com/apache/incubator-seatunnel/pull/3950) +- [Improve] Support read debezium format message [3981](https://github.com/apache/incubator-seatunnel/pull/3981) diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index 06f60af6d879..2ed6ec6f12e4 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -73,8 +73,10 @@ The structure of the data, including field names and field types. ## format -Data format. The default format is json. Optional text format. The default field separator is ", ". -If you customize the delimiter, add the "field_delimiter" option. +Data format. The default format is json. Optional text format, canal-json and debezium-json. +If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option. +If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details. +If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. ## format_error_handle_way @@ -221,9 +223,10 @@ source { ### Next Version -- [Improve] Support setting read starting offset or time at startup config ([3157](https://github.com/apache/seatunnel/pull/3157)) -- [Improve] Support for dynamic discover topic & partition in streaming mode ([3125](https://github.com/apache/seatunnel/pull/3125)) -- [Improve] Change Connector Custom Config Prefix To Map [3719](https://github.com/apache/seatunnel/pull/3719) -- [Bug] Fixed the problem that parsing the offset format failed when the startup mode was offset([3810](https://github.com/apache/seatunnel/pull/3810)) -- [Feature] Kafka source supports data deserialization failure skipping([4364](https://github.com/apache/seatunnel/pull/4364)) +- [Improve] Support setting read starting offset or time at startup config ([3157](https://github.com/apache/incubator-seatunnel/pull/3157)) +- [Improve] Support for dynamic discover topic & partition in streaming mode ([3125](https://github.com/apache/incubator-seatunnel/pull/3125)) +- [Improve] Change Connector Custom Config Prefix To Map [3719](https://github.com/apache/incubator-seatunnel/pull/3719) +- [Bug] Fixed the problem that parsing the offset format failed when the startup mode was offset([3810](https://github.com/apache/incubator-seatunnel/pull/3810)) +- [Improve] Support read canal format message [3950](https://github.com/apache/incubator-seatunnel/pull/3950) +- [Improve] Support read debezium format message [3981](https://github.com/apache/incubator-seatunnel/pull/3981) diff --git a/release-note.md b/release-note.md index 68d14e609f31..0e84da433c25 100644 --- a/release-note.md +++ b/release-note.md @@ -3,9 +3,19 @@ ## Bug fix ### Core - - [Core] [API] Fixed generic class loss for lists (#4421) - [Core] [API] Fix parse nested row data type key changed upper (#4459) +- [Starter][Flink]Support transform-v2 for flink #3396 +- [Flink] Support flink 1.14.x #3963 +### Transformer +- [Spark] Support transform-v2 for spark (#3409) +- [ALL]Add FieldMapper Transform #3781 +### Connectors +- [Elasticsearch] Support https protocol & compatible with opensearch +- [Hbase] Add hbase sink connector #4049 +### Formats +- [Canal]Support read canal format message #3950 +- [Debezium]Support debezium canal format message #3981 ### Connector-V2 diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java index 2dffda4f48ac..f126e563fbb8 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/Config.java @@ -26,8 +26,6 @@ public class Config { public static final String CONNECTOR_IDENTITY = "Kafka"; - public static final String REPLICATION_FACTOR = "replication.factor"; - /** The default field delimiter is “,” */ public static final String DEFAULT_FIELD_DELIMITER = ","; @@ -99,6 +97,12 @@ public class Config { "Data format. The default format is json. Optional text format. The default field separator is \", \". " + "If you customize the delimiter, add the \"field_delimiter\" option."); + public static final Option DEBEZIUM_RECORD_INCLUDE_SCHEMA = + Options.key("debezium_record_include_schema") + .booleanType() + .defaultValue(true) + .withDescription("Does the debezium record carry a schema."); + public static final Option FIELD_DELIMITER = Options.key("field_delimiter") .stringType() diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java index 65b5cc27699c..1ef29f6322a3 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/config/MessageFormat.java @@ -21,5 +21,6 @@ public enum MessageFormat { JSON, TEXT, CANAL_JSON, + DEBEZIUM_JSON, COMPATIBLE_DEBEZIUM_JSON } diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java index 06005de00353..f8974d0f1a9e 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/serialize/DefaultSeaTunnelRowSerializer.java @@ -28,6 +28,7 @@ import org.apache.seatunnel.format.compatible.debezium.json.CompatibleDebeziumJsonSerializationSchema; import org.apache.seatunnel.format.json.JsonSerializationSchema; import org.apache.seatunnel.format.json.canal.CanalJsonSerializationSchema; +import org.apache.seatunnel.format.json.debezium.DebeziumJsonSerializationSchema; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import org.apache.seatunnel.format.text.TextSerializationSchema; @@ -219,6 +220,8 @@ private static SerializationSchema createSerializationSchema( .build(); case CANAL_JSON: return new CanalJsonSerializationSchema(rowType); + case DEBEZIUM_JSON: + return new DebeziumJsonSerializationSchema(rowType); case COMPATIBLE_DEBEZIUM_JSON: return new CompatibleDebeziumJsonSerializationSchema(rowType, isKey); default: diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java index 741d75216439..30878e82a2c4 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSource.java @@ -47,6 +47,7 @@ import org.apache.seatunnel.connectors.seatunnel.kafka.state.KafkaSourceState; import org.apache.seatunnel.format.json.JsonDeserializationSchema; import org.apache.seatunnel.format.json.canal.CanalJsonDeserializationSchema; +import org.apache.seatunnel.format.json.debezium.DebeziumJsonDeserializationSchema; import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; import org.apache.seatunnel.format.text.TextDeserializationSchema; import org.apache.seatunnel.format.text.constant.TextFormatConstant; @@ -62,6 +63,7 @@ import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.BOOTSTRAP_SERVERS; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.COMMIT_ON_CHECKPOINT; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.CONSUMER_GROUP; +import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.DEBEZIUM_RECORD_INCLUDE_SCHEMA; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.DEFAULT_FIELD_DELIMITER; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.FIELD_DELIMITER; import static org.apache.seatunnel.connectors.seatunnel.kafka.config.Config.FORMAT; @@ -266,6 +268,14 @@ private void setDeserialization(Config config) { .setIgnoreParseErrors(true) .build(); break; + case DEBEZIUM_JSON: + boolean includeSchema = DEBEZIUM_RECORD_INCLUDE_SCHEMA.defaultValue(); + if (config.hasPath(DEBEZIUM_RECORD_INCLUDE_SCHEMA.key())) { + includeSchema = config.getBoolean(DEBEZIUM_RECORD_INCLUDE_SCHEMA.key()); + } + deserializationSchema = + new DebeziumJsonDeserializationSchema(typeInfo, true, includeSchema); + break; default: throw new SeaTunnelJsonFormatException( CommonErrorCode.UNSUPPORTED_DATA_TYPE, "Unsupported format: " + format); diff --git a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java index daa75385e4d6..21057040ec2e 100644 --- a/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java +++ b/seatunnel-connectors-v2/connector-kafka/src/main/java/org/apache/seatunnel/connectors/seatunnel/kafka/source/KafkaSourceFactory.java @@ -46,6 +46,7 @@ public OptionRule optionRule() { Config.KAFKA_CONFIG, Config.SCHEMA, Config.FORMAT, + Config.DEBEZIUM_RECORD_INCLUDE_SCHEMA, Config.KEY_PARTITION_DISCOVERY_INTERVAL_MILLIS) .conditional(Config.START_MODE, StartMode.TIMESTAMP, Config.START_MODE_TIMESTAMP) .conditional( diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/DebeziumToKafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/DebeziumToKafkaIT.java new file mode 100644 index 000000000000..e76a4459963c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/DebeziumToKafkaIT.java @@ -0,0 +1,418 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.connector.kafka; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlContainer; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlVersion; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.shaded.org.awaitility.Awaitility; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; +import org.testcontainers.utility.MountableFile; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.net.Inet4Address; +import java.net.InetAddress; +import java.net.NetworkInterface; +import java.net.SocketException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Enumeration; +import java.util.HashSet; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.given; +import static org.testcontainers.shaded.org.awaitility.Awaitility.await; + +@DisabledOnContainer( + value = {}, + type = {EngineType.SEATUNNEL, EngineType.SPARK}) +@Slf4j +public class DebeziumToKafkaIT extends TestSuiteBase implements TestResource { + + private static final Logger LOG = LoggerFactory.getLogger(DebeziumToKafkaIT.class); + + private static GenericContainer DEBEZIUM_CONTAINER; + + private static final String DEBEZIUM_DOCKER_IMAGE = "quay.io/debezium/connect:2.3.0.Final"; + + private static final String DEBEZIUM_HOST = "debezium_e2e"; + + private static final int DEBEZIUM_PORT = 8083; + + // ----------------------------------------kafka------------------------------------ + private static final String KAFKA_IMAGE_NAME = "confluentinc/cp-kafka:7.0.9"; + private static final String KAFKA_HOST = "kafka_dbz_e2e"; + private KafkaConsumer kafkaConsumer; + private KafkaContainer KAFKA_CONTAINER; + private String KAFKA_TOPIC = "test-debezium-sink"; + + // -------------------------------------mysql--------------------------------------- + private static final String MYSQL_HOST = "mysql"; + private static MySqlContainer MYSQL_CONTAINER; + + // -----------------------------------------postgres----------------------------------- + private static final String PG_IMAGE = "postgres:alpine3.16"; + + private static final int PG_PORT = 5432; + + private static final String PG_DRIVER_JAR = + "https://repo1.maven.org/maven2/org/postgresql/postgresql/42.3.3/postgresql-42.3.3.jar"; + + private static PostgreSQLContainer POSTGRESQL_CONTAINER; + + @TestContainerExtension + private final ContainerExtendedFactory extendedFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && curl -O " + + PG_DRIVER_JAR); + Assertions.assertEquals(0, extraCommands.getExitCode()); + }; + + private void createDebeziumContainer() { + DEBEZIUM_CONTAINER = + new GenericContainer<>(DEBEZIUM_DOCKER_IMAGE) + .withCopyFileToContainer( + MountableFile.forClasspathResource("/debezium/register-mysql.json"), + "/tmp/seatunnel/plugins/Jdbc/register-mysql.json") + .withNetwork(NETWORK) + .withNetworkAliases(DEBEZIUM_HOST) + .withExposedPorts(DEBEZIUM_PORT) + .withEnv("GROUP_ID", "1") + .withEnv("CONFIG_STORAGE_TOPIC", "my-connect-configs") + .withEnv("OFFSET_STORAGE_TOPIC", "my-connect-offsets") + .withEnv("STATUS_STORAGE_TOPIC", "my-connect-status") + .withEnv("BOOTSTRAP_SERVERS", KAFKA_HOST + ":9092") + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(DEBEZIUM_DOCKER_IMAGE))) + .dependsOn(KAFKA_CONTAINER, MYSQL_CONTAINER); + DEBEZIUM_CONTAINER.setWaitStrategy( + (new HttpWaitStrategy()) + .forPath("/connectors") + .forPort(DEBEZIUM_PORT) + .withStartupTimeout(Duration.ofSeconds(120))); + DEBEZIUM_CONTAINER.setPortBindings( + com.google.common.collect.Lists.newArrayList( + String.format("%s:%s", DEBEZIUM_PORT, DEBEZIUM_PORT))); + } + + private void createKafkaContainer() { + KAFKA_CONTAINER = + new KafkaContainer(DockerImageName.parse(KAFKA_IMAGE_NAME)) + .withNetwork(NETWORK) + .withNetworkAliases(KAFKA_HOST) + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(KAFKA_IMAGE_NAME))); + } + + private void createMysqlContainer() { + MYSQL_CONTAINER = + new MySqlContainer(MySqlVersion.V8_0) + .withConfigurationOverride("docker/server-gtids/my.cnf") + .withSetupSQL("docker/setup.sql") + .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_HOST) + .withDatabaseName("debezium") + .withUsername("st_user") + .withPassword("seatunnel") + .withLogConsumer(new Slf4jLogConsumer(LOG)); + } + + private void createPostgreSQLContainer() { + POSTGRESQL_CONTAINER = + new PostgreSQLContainer<>(DockerImageName.parse(PG_IMAGE)) + .withNetwork(NETWORK) + .withNetworkAliases("postgresql_e2e") + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(PG_IMAGE))); + POSTGRESQL_CONTAINER.setPortBindings( + Lists.newArrayList(String.format("%s:%s", PG_PORT, PG_PORT))); + } + + @BeforeAll + @Override + public void startUp() throws Exception { + LOG.info("The first stage: Starting Kafka containers..."); + createKafkaContainer(); + Startables.deepStart(Stream.of(KAFKA_CONTAINER)).join(); + + LOG.info("The second stage: Starting Mysql containers..."); + createMysqlContainer(); + Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); + + LOG.info("The third stage: Starting Debezium Connector containers..."); + createDebeziumContainer(); + Startables.deepStart(Stream.of(DEBEZIUM_CONTAINER)).join(); + + LOG.info("The fourth stage: Starting PostgreSQL container..."); + createPostgreSQLContainer(); + Startables.deepStart(Stream.of(POSTGRESQL_CONTAINER)).join(); + Class.forName(POSTGRESQL_CONTAINER.getDriverClassName()); + + Awaitility.given() + .ignoreExceptions() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(3, TimeUnit.MINUTES) + .untilAsserted(this::initializeSourceTableData); + + given().ignoreExceptions() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(3, TimeUnit.MINUTES) + .untilAsserted(this::initKafkaConsumer); + + given().ignoreExceptions() + .await() + .atLeast(100, TimeUnit.MILLISECONDS) + .pollInterval(500, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.MINUTES) + .untilAsserted(this::initializeSinkJdbcTable); + + Container.ExecResult extraCommand = + DEBEZIUM_CONTAINER.execInContainer( + "bash", + "-c", + "cd /tmp/seatunnel/plugins/Jdbc && curl -i -X POST -H \"Accept:application/json\" -H \"Content-Type:application/json\" http://" + + getLinuxLocalIp() + + ":8083/connectors/ -d @register-mysql.json"); + Assertions.assertEquals(0, extraCommand.getExitCode()); + // ensure debezium has handled the data + Thread.sleep(30 * 1000); + updateSourceTableData(); + Thread.sleep(30 * 1000); + } + + @AfterAll + @Override + public void tearDown() throws Exception { + MYSQL_CONTAINER.close(); + KAFKA_CONTAINER.close(); + DEBEZIUM_CONTAINER.close(); + POSTGRESQL_CONTAINER.close(); + } + + @TestTemplate + public void testKafkaSinkDebeziumFormat(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult execResult = + container.executeJob("/kafkasource_debezium_to_kafka.conf"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + ArrayList result = new ArrayList<>(); + kafkaConsumer.subscribe(Lists.newArrayList(KAFKA_TOPIC)); + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + ConsumerRecords consumerRecords = + kafkaConsumer.poll(Duration.ofMillis(1000)); + for (ConsumerRecord record : consumerRecords) { + result.add(record.value()); + } + Assertions.assertEquals(12, result.size()); + }); + } + + @TestTemplate + public void testDebeziumFormatKafkaCdcToPgsql(TestContainer container) + throws IOException, InterruptedException, SQLException { + Container.ExecResult execResult = + container.executeJob("/kafkasource_debezium_cdc_to_pgsql.conf"); + Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + Set> actual = new HashSet<>(); + try (Connection connection = + DriverManager.getConnection( + POSTGRESQL_CONTAINER.getJdbcUrl(), + POSTGRESQL_CONTAINER.getUsername(), + POSTGRESQL_CONTAINER.getPassword())) { + try (Statement statement = connection.createStatement()) { + ResultSet resultSet = statement.executeQuery("select * from sink order by id"); + while (resultSet.next()) { + List row = + Arrays.asList( + resultSet.getInt("id"), + resultSet.getString("name"), + resultSet.getString("description"), + resultSet.getString("weight")); + actual.add(row); + } + } + } + Set> expected = + Stream.>of( + Arrays.asList(101, "scooter", "Small 2-wheel scooter", "4.56"), + Arrays.asList(102, "car battery", "12V car battery", "8.1"), + Arrays.asList( + 103, + "12-pack drill bits", + "12-pack of drill bits with sizes ranging from #40 to #3", + "0.8"), + Arrays.asList(104, "hammer", "12oz carpenter's hammer", "0.75"), + Arrays.asList(105, "hammer", "14oz carpenter's hammer", "0.875"), + Arrays.asList(106, "hammer", "16oz carpenter's hammer", "1"), + Arrays.asList(107, "rocks", "box of assorted rocks", "5.3"), + Arrays.asList( + 108, "jacket", "water resistent black wind breaker", "0.1")) + .collect(Collectors.toSet()); + Assertions.assertIterableEquals(expected, actual); + } + + public void initializeSourceTableData() throws Exception { + try (Connection connection = + DriverManager.getConnection( + MYSQL_CONTAINER.getJdbcUrl(), + MYSQL_CONTAINER.getUsername(), + MYSQL_CONTAINER.getPassword()); + Statement statement = connection.createStatement()) { + statement.execute("create database if not exists debezium"); + statement.execute( + "CREATE TABLE if not exists debezium.products (\n" + + " id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,\n" + + " name VARCHAR(255) NOT NULL DEFAULT 'SeaTunnel',\n" + + " description VARCHAR(512),\n" + + " weight VARCHAR(512)\n" + + ");"); + statement.execute( + "INSERT INTO debezium.products\n" + + "VALUES (101,\"scooter\",\"Small 2-wheel scooter\",\"3.14\"),\n" + + " (102,\"car battery\",\"12V car battery\",\"8.1\"),\n" + + " (103,\"12-pack drill bits\",\"12-pack of drill bits with sizes ranging from #40 to #3\"," + + "\"0.8\"),\n" + + " (104,\"hammer\",\"12oz carpenter's hammer\",\"0.75\"),\n" + + " (105,\"hammer\",\"14oz carpenter's hammer\",\"0.875\"),\n" + + " (106,\"hammer\",\"16oz carpenter's hammer\",\"1.0\"),\n" + + " (107,\"rocks\",\"box of assorted rocks\",\"5.3\"),\n" + + " (108,\"jacket\",\"water resistent black wind breaker\",\"0.1\"),\n" + + " (109,\"spare tire\",\"24 inch spare tire\",\"22.2\")"); + } + } + + public void updateSourceTableData() throws Exception { + try (Connection connection = + DriverManager.getConnection( + MYSQL_CONTAINER.getJdbcUrl(), + MYSQL_CONTAINER.getUsername(), + MYSQL_CONTAINER.getPassword()); + Statement statement = connection.createStatement()) { + statement.execute( + "UPDATE debezium.products SET weight = '4.56' WHERE name = 'scooter'"); + statement.execute("DELETE FROM debezium.products WHERE name = \"spare tire\""); + } + } + + private void initializeSinkJdbcTable() { + try (Connection connection = + DriverManager.getConnection( + POSTGRESQL_CONTAINER.getJdbcUrl(), + POSTGRESQL_CONTAINER.getUsername(), + POSTGRESQL_CONTAINER.getPassword()); + Statement statement = connection.createStatement()) { + String sink = + "create table sink(\n" + + "id INT NOT NULL PRIMARY KEY,\n" + + "name varchar(255),\n" + + "description varchar(255),\n" + + "weight varchar(255)" + + ")"; + statement.execute(sink); + } catch (SQLException e) { + throw new RuntimeException("Initializing PostgreSql table failed!", e); + } + } + + private void initKafkaConsumer() { + Properties prop = new Properties(); + String bootstrapServers = KAFKA_CONTAINER.getBootstrapServers(); + prop.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + prop.put( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringDeserializer"); + prop.put( + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, + "org.apache.kafka.common.serialization.StringDeserializer"); + prop.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + prop.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, true); + prop.put(ConsumerConfig.GROUP_ID_CONFIG, "seatunnel-debezium-sink-group"); + kafkaConsumer = new KafkaConsumer<>(prop); + } + + public String getLinuxLocalIp() { + String ip = ""; + try { + Enumeration networkInterfaces = + NetworkInterface.getNetworkInterfaces(); + while (networkInterfaces.hasMoreElements()) { + NetworkInterface networkInterface = networkInterfaces.nextElement(); + Enumeration inetAddresses = networkInterface.getInetAddresses(); + while (inetAddresses.hasMoreElements()) { + InetAddress inetAddress = inetAddresses.nextElement(); + if (!inetAddress.isLoopbackAddress() && inetAddress instanceof Inet4Address) { + ip = inetAddress.getHostAddress(); + } + } + } + } catch (SocketException ex) { + log.warn("Failed to get linux local ip, it will return [\"\"] ", ex); + } + return ip; + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/debezium/register-mysql.json b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/debezium/register-mysql.json new file mode 100644 index 000000000000..d70e8e0c613d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/debezium/register-mysql.json @@ -0,0 +1,16 @@ +{ + "name": "inventory-connector", + "config": { + "connector.class": "io.debezium.connector.mysql.MySqlConnector", + "tasks.max": "1", + "database.hostname": "mysql", + "database.port": "3306", + "database.user": "st_user", + "database.password": "seatunnel", + "database.server.id": "184054", + "topic.prefix": "dbserver1", + "database.include.list": "debezium", + "schema.history.internal.kafka.bootstrap.servers": "kafka_dbz_e2e:9092", + "schema.history.internal.kafka.topic": "schema-changes.debezium" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_cdc_to_pgsql.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_cdc_to_pgsql.conf new file mode 100644 index 000000000000..a0531b2345a0 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_cdc_to_pgsql.conf @@ -0,0 +1,62 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + execution.parallelism = 1 + job.mode = "BATCH" + + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + Kafka { + bootstrap.servers = "kafka_dbz_e2e:9092" + topic = "dbserver1.debezium.products" + result_table_name = "kafka_name" + start_mode = earliest + format = debezium_json + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "float" + } + } + } +} + +sink { + Jdbc { + driver = org.postgresql.Driver + url = "jdbc:postgresql://postgresql_e2e:5432/test?loggerLevel=OFF" + user = test + password = test + generate_sink_sql = true + database = public + table = sink + primary_keys = ["id"] + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_to_kafka.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_to_kafka.conf new file mode 100644 index 000000000000..4944829c24ab --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_to_kafka.conf @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + execution.parallelism = 1 + job.mode = "BATCH" + + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + Kafka { + bootstrap.servers = "kafka_dbz_e2e:9092" + topic = "dbserver1.debezium.products" + result_table_name = "kafka_name" + start_mode = earliest + format = debezium_json + schema = { + fields { + id = "int" + name = "string" + description = "string" + weight = "float" + } + } + } +} + +sink { + Kafka { + bootstrap.servers = "kafka_dbz_e2e:9092" + topic = "test-debezium-sink" + format = debezium_json + } +} \ No newline at end of file diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonFormatOptions.java index 7b10ad57a634..9ce4dc554118 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonFormatOptions.java +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/JsonFormatOptions.java @@ -24,6 +24,12 @@ import java.util.Map; public class JsonFormatOptions { + public static final Option FAIL_ON_MISSING_FIELD = + Options.key("fail-on-missing-field") + .booleanType() + .defaultValue(false) + .withDescription( + "Optional flag to specify whether to fail if a field is missing or not, false by default."); public static final Option IGNORE_PARSE_ERRORS = Options.key("ignore-parse-errors") @@ -33,13 +39,6 @@ public class JsonFormatOptions { "Optional flag to skip fields and rows with parse errors instead of failing;\n" + "fields are set to null in case of errors, false by default."); - public static final Option FAIL_ON_MISSING_FIELD = - Options.key("fail-on-missing-field") - .booleanType() - .defaultValue(false) - .withDescription( - "Optional flag to specify whether to fail if a field is missing or not, false by default."); - public static boolean getFailOnMissingField(Map options) { return Boolean.parseBoolean( options.getOrDefault( diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java new file mode 100644 index 000000000000..3996c4ed7d82 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonDeserializationSchema.java @@ -0,0 +1,168 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.format.json.debezium; + +import org.apache.seatunnel.shade.com.fasterxml.jackson.databind.JsonNode; + +import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.format.json.JsonDeserializationSchema; +import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; + +import java.io.IOException; + +public class DebeziumJsonDeserializationSchema implements DeserializationSchema { + private static final long serialVersionUID = 1L; + + private static final String OP_READ = "r"; // snapshot read + private static final String OP_CREATE = "c"; // insert + private static final String OP_UPDATE = "u"; // update + private static final String OP_DELETE = "d"; // delete + + private static final String REPLICA_IDENTITY_EXCEPTION = + "The \"before\" field of %s message is null, " + + "if you are using Debezium Postgres Connector, " + + "please check the Postgres table has been set REPLICA IDENTITY to FULL level."; + + private final SeaTunnelRowType rowType; + + private final JsonDeserializationSchema jsonDeserializer; + + private final boolean ignoreParseErrors; + + private final boolean debeziumEnabledSchema; + + public DebeziumJsonDeserializationSchema(SeaTunnelRowType rowType, boolean ignoreParseErrors) { + this.rowType = rowType; + this.ignoreParseErrors = ignoreParseErrors; + this.jsonDeserializer = + new JsonDeserializationSchema(false, ignoreParseErrors, createJsonRowType(rowType)); + this.debeziumEnabledSchema = false; + } + + public DebeziumJsonDeserializationSchema( + SeaTunnelRowType rowType, boolean ignoreParseErrors, boolean debeziumEnabledSchema) { + this.rowType = rowType; + this.ignoreParseErrors = ignoreParseErrors; + this.jsonDeserializer = + new JsonDeserializationSchema(false, ignoreParseErrors, createJsonRowType(rowType)); + this.debeziumEnabledSchema = debeziumEnabledSchema; + } + + @Override + public SeaTunnelRow deserialize(byte[] message) throws IOException { + throw new UnsupportedOperationException( + "Please invoke DeserializationSchema#deserialize(byte[], Collector) instead."); + } + + @Override + public void deserialize(byte[] message, Collector out) throws IOException { + if (message == null || message.length == 0) { + // skip tombstone messages + return; + } + + try { + JsonNode payload = getPayload(convertBytes(message)); + String op = payload.get("op").asText(); + + if (OP_CREATE.equals(op) || OP_READ.equals(op)) { + SeaTunnelRow insert = convertJsonNode(payload.get("after")); + insert.setRowKind(RowKind.INSERT); + out.collect(insert); + } else if (OP_UPDATE.equals(op)) { + SeaTunnelRow before = convertJsonNode(payload.get("before")); + if (before == null) { + throw new SeaTunnelJsonFormatException( + CommonErrorCode.UNSUPPORTED_DATA_TYPE, + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + before.setRowKind(RowKind.UPDATE_BEFORE); + out.collect(before); + + SeaTunnelRow after = convertJsonNode(payload.get("after")); + after.setRowKind(RowKind.UPDATE_AFTER); + out.collect(after); + } else if (OP_DELETE.equals(op)) { + SeaTunnelRow delete = convertJsonNode(payload.get("before")); + if (delete == null) { + throw new SeaTunnelJsonFormatException( + CommonErrorCode.UNSUPPORTED_DATA_TYPE, + String.format(REPLICA_IDENTITY_EXCEPTION, "UPDATE")); + } + delete.setRowKind(RowKind.DELETE); + out.collect(delete); + } else { + if (!ignoreParseErrors) { + throw new SeaTunnelJsonFormatException( + CommonErrorCode.UNSUPPORTED_DATA_TYPE, + String.format( + "Unknown \"op\" value \"%s\". The Debezium JSON message is '%s'", + op, new String(message))); + } + } + } catch (Throwable t) { + // a big try catch to protect the processing. + if (!ignoreParseErrors) { + throw new SeaTunnelJsonFormatException( + CommonErrorCode.UNSUPPORTED_DATA_TYPE, + String.format("Corrupt Debezium JSON message '%s'.", new String(message)), + t); + } + } + } + + private JsonNode getPayload(JsonNode jsonNode) { + if (debeziumEnabledSchema) { + return jsonNode.get("payload"); + } + return jsonNode; + } + + private JsonNode convertBytes(byte[] message) { + try { + return jsonDeserializer.deserializeToJsonNode(message); + } catch (Exception t) { + if (ignoreParseErrors) { + return null; + } + throw new SeaTunnelJsonFormatException( + CommonErrorCode.JSON_OPERATION_FAILED, + String.format("Failed to deserialize JSON '%s'.", new String(message)), + t); + } + } + + private SeaTunnelRow convertJsonNode(JsonNode root) { + return jsonDeserializer.convertToRowData(root); + } + + @Override + public SeaTunnelDataType getProducedType() { + return this.rowType; + } + + private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType databaseSchema) { + return databaseSchema; + } +} diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatFactory.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatFactory.java new file mode 100644 index 000000000000..e59c9794fb64 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatFactory.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.format.json.debezium; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.serialization.DeserializationSchema; +import org.apache.seatunnel.api.serialization.SerializationSchema; +import org.apache.seatunnel.api.table.connector.DeserializationFormat; +import org.apache.seatunnel.api.table.connector.SerializationFormat; +import org.apache.seatunnel.api.table.factory.DeserializationFormatFactory; +import org.apache.seatunnel.api.table.factory.SerializationFormatFactory; +import org.apache.seatunnel.api.table.factory.TableFactoryContext; + +import java.util.Map; + +public class DebeziumJsonFormatFactory + implements DeserializationFormatFactory, SerializationFormatFactory { + + public static final String IDENTIFIER = "debezium_json"; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder().build(); + } + + @Override + public SerializationFormat createSerializationFormat(TableFactoryContext context) { + return new SerializationFormat() { + @Override + public SerializationSchema createSerializationSchema() { + return new DebeziumJsonSerializationSchema(null); + } + }; + } + + @Override + public DeserializationFormat createDeserializationFormat(TableFactoryContext context) { + Map options = context.getOptions().toMap(); + boolean ignoreParseErrors = DebeziumJsonFormatOptions.getIgnoreParseErrors(options); + boolean schemaInclude = DebeziumJsonFormatOptions.getSchemaInclude(options); + + // TODO config SeaTunnelRowType + return new DeserializationFormat() { + @Override + public DeserializationSchema createDeserializationSchema() { + return new DebeziumJsonDeserializationSchema(null, ignoreParseErrors); + } + }; + } +} diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java new file mode 100644 index 000000000000..eb75bfd2b0cf --- /dev/null +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonFormatOptions.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.format.json.debezium; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.format.json.JsonFormatOptions; + +import java.util.Map; + +public class DebeziumJsonFormatOptions { + + public static final int GENERATE_ROW_SIZE = 3; + + public static final Option IGNORE_PARSE_ERRORS = JsonFormatOptions.IGNORE_PARSE_ERRORS; + + public static final Option SCHEMA_INCLUDE = + Options.key("schema-include") + .booleanType() + .defaultValue(false) + .withDescription( + "When setting up a Debezium Kafka Connect, users can enable " + + "a Kafka configuration 'value.converter.schemas.enable' to include schema in the message. " + + "This option indicates the Debezium JSON data include the schema in the message or not. " + + "Default is false."); + + public static boolean getSchemaInclude(Map options) { + return Boolean.parseBoolean( + options.getOrDefault( + SCHEMA_INCLUDE.key(), SCHEMA_INCLUDE.defaultValue().toString())); + } + + public static boolean getIgnoreParseErrors(Map options) { + return Boolean.parseBoolean( + options.getOrDefault( + IGNORE_PARSE_ERRORS.key(), IGNORE_PARSE_ERRORS.defaultValue().toString())); + } +} diff --git a/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java new file mode 100644 index 000000000000..5b1e476abcd8 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-json/src/main/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerializationSchema.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.format.json.debezium; + +import org.apache.seatunnel.api.serialization.SerializationSchema; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.format.json.JsonSerializationSchema; +import org.apache.seatunnel.format.json.exception.SeaTunnelJsonFormatException; + +import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; +import static org.apache.seatunnel.format.json.debezium.DebeziumJsonFormatOptions.GENERATE_ROW_SIZE; + +public class DebeziumJsonSerializationSchema implements SerializationSchema { + private static final long serialVersionUID = 1L; + + private static final String OP_INSERT = "c"; // insert + private static final String OP_DELETE = "d"; // delete + + private final JsonSerializationSchema jsonSerializer; + + private transient SeaTunnelRow genericRow; + + public DebeziumJsonSerializationSchema(SeaTunnelRowType rowType) { + this.jsonSerializer = new JsonSerializationSchema(createJsonRowType(rowType)); + this.genericRow = new SeaTunnelRow(GENERATE_ROW_SIZE); + } + + @Override + public byte[] serialize(SeaTunnelRow row) { + try { + switch (row.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + genericRow.setField(0, null); + genericRow.setField(1, row); + genericRow.setField(2, OP_INSERT); + return jsonSerializer.serialize(genericRow); + case UPDATE_BEFORE: + case DELETE: + genericRow.setField(0, row); + genericRow.setField(1, null); + genericRow.setField(2, OP_DELETE); + return jsonSerializer.serialize(genericRow); + default: + throw new UnsupportedOperationException( + String.format( + "Unsupported operation '%s' for row kind.", row.getRowKind())); + } + } catch (Throwable t) { + throw new SeaTunnelJsonFormatException( + CommonErrorCode.JSON_OPERATION_FAILED, + String.format("Could not serialize row %s.", row), + t); + } + } + + private static SeaTunnelRowType createJsonRowType(SeaTunnelRowType databaseSchema) { + return new SeaTunnelRowType( + new String[] {"before", "after", "op"}, + new SeaTunnelDataType[] {databaseSchema, databaseSchema, STRING_TYPE}); + } +} diff --git a/seatunnel-formats/seatunnel-format-json/src/main/resources/META-INF/services/org.apache.seatunnel.api.table.factory.Factory b/seatunnel-formats/seatunnel-format-json/src/main/resources/META-INF/services/org.apache.seatunnel.api.table.factory.Factory index db11c51c4a2d..cedeba7515d2 100644 --- a/seatunnel-formats/seatunnel-format-json/src/main/resources/META-INF/services/org.apache.seatunnel.api.table.factory.Factory +++ b/seatunnel-formats/seatunnel-format-json/src/main/resources/META-INF/services/org.apache.seatunnel.api.table.factory.Factory @@ -17,3 +17,4 @@ org.apache.seatunnel.format.json.JsonFormatFactory org.apache.seatunnel.format.json.canal.CanalJsonFormatFactory +org.apache.seatunnel.format.json.debezium.DebeziumJsonFormatFactory diff --git a/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java new file mode 100644 index 000000000000..20088e525bfb --- /dev/null +++ b/seatunnel-formats/seatunnel-format-json/src/test/java/org/apache/seatunnel/format/json/debezium/DebeziumJsonSerDeSchemaTest.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.format.json.debezium; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.api.table.type.BasicType.FLOAT_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.INT_TYPE; +import static org.apache.seatunnel.api.table.type.BasicType.STRING_TYPE; +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class DebeziumJsonSerDeSchemaTest { + + private static final SeaTunnelRowType PHYSICAL_DATA_TYPE = + new SeaTunnelRowType( + new String[] {"id", "name", "description", "weight"}, + new SeaTunnelDataType[] {INT_TYPE, STRING_TYPE, STRING_TYPE, FLOAT_TYPE}); + + @Test + void testNullRowMessages() throws Exception { + DebeziumJsonDeserializationSchema deserializationSchema = + new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, false); + SimpleCollector collector = new SimpleCollector(); + + deserializationSchema.deserialize(null, collector); + deserializationSchema.deserialize(new byte[0], collector); + assertEquals(0, collector.list.size()); + } + + @Test + public void testSerializationAndSchemaExcludeDeserialization() throws Exception { + testSerializationDeserialization("debezium-data.txt", false); + } + + private void testSerializationDeserialization(String resourceFile, boolean schemaInclude) + throws Exception { + List lines = readLines(resourceFile); + DebeziumJsonDeserializationSchema deserializationSchema = + new DebeziumJsonDeserializationSchema(PHYSICAL_DATA_TYPE, true, schemaInclude); + + SimpleCollector collector = new SimpleCollector(); + + for (String line : lines) { + deserializationSchema.deserialize(line.getBytes(StandardCharsets.UTF_8), collector); + } + + List expected = + Arrays.asList( + "SeaTunnelRow{tableId=, kind=+I, fields=[101, scooter, Small 2-wheel scooter, 3.14]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[102, car battery, 12V car battery, 8.1]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[104, hammer, 12oz carpenter's hammer, 0.75]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[105, hammer, 14oz carpenter's hammer, 0.875]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[108, jacket, water resistent black wind breaker, 0.1]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[109, spare tire, 24 inch spare tire, 22.2]}", + "SeaTunnelRow{tableId=, kind=-U, fields=[106, hammer, 16oz carpenter's hammer, 1.0]}", + "SeaTunnelRow{tableId=, kind=+U, fields=[106, hammer, 18oz carpenter hammer, 1.0]}", + "SeaTunnelRow{tableId=, kind=-U, fields=[107, rocks, box of assorted rocks, 5.3]}", + "SeaTunnelRow{tableId=, kind=+U, fields=[107, rocks, box of assorted rocks, 5.1]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=, kind=+I, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=, kind=-U, fields=[110, jacket, water resistent white wind breaker, 0.2]}", + "SeaTunnelRow{tableId=, kind=+U, fields=[110, jacket, new water resistent white wind breaker, 0.5]}", + "SeaTunnelRow{tableId=, kind=-U, fields=[111, scooter, Big 2-wheel scooter , 5.18]}", + "SeaTunnelRow{tableId=, kind=+U, fields=[111, scooter, Big 2-wheel scooter , 5.17]}", + "SeaTunnelRow{tableId=, kind=-D, fields=[111, scooter, Big 2-wheel scooter , 5.17]}"); + List actual = + collector.list.stream().map(Object::toString).collect(Collectors.toList()); + assertEquals(expected, actual); + + DebeziumJsonSerializationSchema serializationSchema = + new DebeziumJsonSerializationSchema(PHYSICAL_DATA_TYPE); + + actual = new ArrayList<>(); + for (SeaTunnelRow rowData : collector.list) { + actual.add(new String(serializationSchema.serialize(rowData), StandardCharsets.UTF_8)); + } + + expected = + Arrays.asList( + "{\"before\":null,\"after\":{\"id\":101,\"name\":\"scooter\",\"description\":\"Small 2-wheel scooter\",\"weight\":3.14},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":102,\"name\":\"car battery\",\"description\":\"12V car battery\",\"weight\":8.1},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":103,\"name\":\"12-pack drill bits\",\"description\":\"12-pack of drill bits with sizes ranging from #40 to #3\",\"weight\":0.8},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":104,\"name\":\"hammer\",\"description\":\"12oz carpenter's hammer\",\"weight\":0.75},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":105,\"name\":\"hammer\",\"description\":\"14oz carpenter's hammer\",\"weight\":0.875},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":108,\"name\":\"jacket\",\"description\":\"water resistent black wind breaker\",\"weight\":0.1},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":109,\"name\":\"spare tire\",\"description\":\"24 inch spare tire\",\"weight\":22.2},\"op\":\"c\"}", + "{\"before\":{\"id\":106,\"name\":\"hammer\",\"description\":\"16oz carpenter's hammer\",\"weight\":1.0},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":106,\"name\":\"hammer\",\"description\":\"18oz carpenter hammer\",\"weight\":1.0},\"op\":\"c\"}", + "{\"before\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.3},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":107,\"name\":\"rocks\",\"description\":\"box of assorted rocks\",\"weight\":5.1},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"op\":\"c\"}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"op\":\"c\"}", + "{\"before\":{\"id\":110,\"name\":\"jacket\",\"description\":\"water resistent white wind breaker\",\"weight\":0.2},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":110,\"name\":\"jacket\",\"description\":\"new water resistent white wind breaker\",\"weight\":0.5},\"op\":\"c\"}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.18},\"after\":null,\"op\":\"d\"}", + "{\"before\":null,\"after\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"op\":\"c\"}", + "{\"before\":{\"id\":111,\"name\":\"scooter\",\"description\":\"Big 2-wheel scooter \",\"weight\":5.17},\"after\":null,\"op\":\"d\"}"); + assertEquals(expected, actual); + } + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + private static List readLines(String resource) throws IOException { + final URL url = DebeziumJsonSerDeSchemaTest.class.getClassLoader().getResource(resource); + Assertions.assertNotNull(url); + Path path = new File(url.getFile()).toPath(); + return Files.readAllLines(path); + } + + private static class SimpleCollector implements Collector { + + private List list = new ArrayList<>(); + + @Override + public void collect(SeaTunnelRow record) { + list.add(record); + } + + @Override + public Object getCheckpointLock() { + return null; + } + } +} diff --git a/seatunnel-formats/seatunnel-format-json/src/test/resources/debezium-data.txt b/seatunnel-formats/seatunnel-format-json/src/test/resources/debezium-data.txt new file mode 100644 index 000000000000..3763369e4987 --- /dev/null +++ b/seatunnel-formats/seatunnel-format-json/src/test/resources/debezium-data.txt @@ -0,0 +1,16 @@ +{"before":null,"after":{"id":101,"name":"scooter","description":"Small 2-wheel scooter","weight":3.140000104904175},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606100,"transaction":null} +{"before":null,"after":{"id":102,"name":"car battery","description":"12V car battery","weight":8.100000381469727},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":103,"name":"12-pack drill bits","description":"12-pack of drill bits with sizes ranging from #40 to #3","weight":0.800000011920929},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":104,"name":"hammer","description":"12oz carpenter's hammer","weight":0.75},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":105,"name":"hammer","description":"14oz carpenter's hammer","weight":0.875},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":108,"name":"jacket","description":"water resistent black wind breaker","weight":0.10000000149011612},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":null,"after":{"id":109,"name":"spare tire","description":"24 inch spare tire","weight":22.200000762939453},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":0,"snapshot":"true","db":"inventory","table":"products","server_id":0,"gtid":null,"file":"mysql-bin.000003","pos":154,"row":0,"thread":null,"query":null},"op":"c","ts_ms":1589355606101,"transaction":null} +{"before":{"id":106,"name":"hammer","description":"16oz carpenter's hammer","weight":1},"after":{"id":106,"name":"hammer","description":"18oz carpenter hammer","weight":1},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589361987000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":362,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589361987936,"transaction":null} +{"before":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.300000190734863},"after":{"id":107,"name":"rocks","description":"box of assorted rocks","weight":5.099999904632568},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362099000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":717,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362099505,"transaction":null} +{"before":null,"after":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362210000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1068,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362210230,"transaction":null} +{"before":null,"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362243000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1394,"row":0,"thread":2,"query":null},"op":"c","ts_ms":1589362243428,"transaction":null} +{"before":{"id":110,"name":"jacket","description":"water resistent white wind breaker","weight":0.20000000298023224},"after":{"id":110,"name":"jacket","description":"new water resistent white wind breaker","weight":0.5},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362293000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":1707,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362293539,"transaction":null} +{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.179999828338623},"after":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362330000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2090,"row":0,"thread":2,"query":null},"op":"u","ts_ms":1589362330904,"transaction":null} +{"before":{"id":111,"name":"scooter","description":"Big 2-wheel scooter ","weight":5.170000076293945},"after":null,"source":{"version":"1.1.1.Final","connector":"mysql","name":"dbserver1","ts_ms":1589362344000,"snapshot":"false","db":"inventory","table":"products","server_id":223344,"gtid":null,"file":"mysql-bin.000003","pos":2443,"row":0,"thread":2,"query":null},"op":"d","ts_ms":1589362344455,"transaction":null} \ No newline at end of file From 32b7f2b6906cbbd7c177ca00fabde974d384f06f Mon Sep 17 00:00:00 2001 From: hailin0 Date: Wed, 26 Jul 2023 10:25:57 +0800 Subject: [PATCH 19/43] [Feature][CDC] Support tables without primary keys (with unique keys) (#163) (#5150) --- .../base/dialect/JdbcDataSourceDialect.java | 100 ++++++++++++++++++ .../splitter/JdbcSourceChunkSplitter.java | 44 ++++++++ .../source/eumerator/MySqlChunkSplitter.java | 18 +--- .../source/source/SqlServerDialect.java | 5 +- .../eumerator/SqlServerChunkSplitter.java | 18 +--- .../source/utils/SqlServerSchema.java | 18 ++-- .../jdbc/catalog/AbstractJdbcCatalog.java | 10 +- .../seatunnel/jdbc/sink/JdbcSinkFactory.java | 17 +++ 8 files changed, 181 insertions(+), 49 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/dialect/JdbcDataSourceDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/dialect/JdbcDataSourceDialect.java index 2c93bf387a60..17947ad1a6bf 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/dialect/JdbcDataSourceDialect.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/dialect/JdbcDataSourceDialect.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.connectors.cdc.base.dialect; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionFactory; @@ -25,11 +27,23 @@ import org.apache.seatunnel.connectors.cdc.base.source.reader.external.JdbcSourceFetchTaskContext; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.tuple.Pair; + import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; public interface JdbcDataSourceDialect extends DataSourceDialect { @@ -68,4 +82,90 @@ default JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { @Override JdbcSourceFetchTaskContext createFetchTaskContext( SourceSplitBase sourceSplitBase, JdbcSourceConfig taskSourceConfig); + + default Optional getPrimaryKey(JdbcConnection jdbcConnection, TableId tableId) + throws SQLException { + + DatabaseMetaData metaData = jdbcConnection.connection().getMetaData(); + + // According to the Javadoc of java.sql.DatabaseMetaData#getPrimaryKeys, + // the returned primary key columns are ordered by COLUMN_NAME, not by KEY_SEQ. + // We need to sort them based on the KEY_SEQ value. + ResultSet rs = + metaData.getPrimaryKeys(tableId.catalog(), tableId.schema(), tableId.table()); + + // seq -> column name + List> primaryKeyColumns = new ArrayList<>(); + String pkName = null; + while (rs.next()) { + // all the PK_NAME should be the same + pkName = rs.getString("PK_NAME"); + String columnName = rs.getString("COLUMN_NAME"); + int keySeq = rs.getInt("KEY_SEQ"); + // KEY_SEQ is 1-based index + primaryKeyColumns.add(Pair.of(keySeq, columnName)); + } + // initialize size + List pkFields = + primaryKeyColumns.stream() + .sorted(Comparator.comparingInt(Pair::getKey)) + .map(Pair::getValue) + .collect(Collectors.toList()); + if (CollectionUtils.isEmpty(pkFields)) { + return Optional.empty(); + } + return Optional.of(PrimaryKey.of(pkName, pkFields)); + } + + default List getUniqueKeys(JdbcConnection jdbcConnection, TableId tableId) + throws SQLException { + return getConstraintKeys(jdbcConnection, tableId).stream() + .filter( + constraintKey -> + constraintKey.getConstraintType() + == ConstraintKey.ConstraintType.UNIQUE_KEY) + .collect(Collectors.toList()); + } + + default List getConstraintKeys(JdbcConnection jdbcConnection, TableId tableId) + throws SQLException { + DatabaseMetaData metaData = jdbcConnection.connection().getMetaData(); + + ResultSet resultSet = + metaData.getIndexInfo( + tableId.catalog(), tableId.schema(), tableId.table(), false, false); + // index name -> index + Map constraintKeyMap = new HashMap<>(); + while (resultSet.next()) { + String columnName = resultSet.getString("COLUMN_NAME"); + if (columnName == null) { + continue; + } + + String indexName = resultSet.getString("INDEX_NAME"); + boolean noUnique = resultSet.getBoolean("NON_UNIQUE"); + + ConstraintKey constraintKey = + constraintKeyMap.computeIfAbsent( + indexName, + s -> { + ConstraintKey.ConstraintType constraintType = + ConstraintKey.ConstraintType.KEY; + if (!noUnique) { + constraintType = ConstraintKey.ConstraintType.UNIQUE_KEY; + } + return ConstraintKey.of( + constraintType, indexName, new ArrayList<>()); + }); + + ConstraintKey.ColumnSortType sortType = + "A".equals(resultSet.getString("ASC_OR_DESC")) + ? ConstraintKey.ColumnSortType.ASC + : ConstraintKey.ColumnSortType.DESC; + ConstraintKey.ConstraintKeyColumn constraintKeyColumn = + new ConstraintKey.ConstraintKeyColumn(columnName, sortType); + constraintKey.getColumnNames().add(constraintKeyColumn); + } + return new ArrayList<>(constraintKeyMap.values()); + } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java index 9e42d5526351..bbad9d04b1e1 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java @@ -17,16 +17,22 @@ package org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Column; +import io.debezium.relational.Table; import io.debezium.relational.TableId; import java.sql.SQLException; import java.util.Collection; +import java.util.List; +import java.util.Optional; /** The {@code ChunkSplitter} used to split table into a set of chunks for JDBC data source. */ public interface JdbcSourceChunkSplitter extends ChunkSplitter { @@ -161,4 +167,42 @@ default SeaTunnelRowType getSplitType(Column splitColumn) { new String[] {splitColumn.name()}, new SeaTunnelDataType[] {fromDbzColumn(splitColumn)}); } + + default Column getSplitColumn( + JdbcConnection jdbc, JdbcDataSourceDialect dialect, TableId tableId) + throws SQLException { + Optional primaryKey = dialect.getPrimaryKey(jdbc, tableId); + if (primaryKey.isPresent()) { + List pkColumns = primaryKey.get().getColumnNames(); + + Table table = dialect.queryTableSchema(jdbc, tableId).getTable(); + for (String pkColumn : pkColumns) { + Column column = table.columnWithName(pkColumn); + if (isEvenlySplitColumn(column)) { + return column; + } + } + } + + List uniqueKeys = dialect.getUniqueKeys(jdbc, tableId); + if (!uniqueKeys.isEmpty()) { + Table table = dialect.queryTableSchema(jdbc, tableId).getTable(); + for (ConstraintKey uniqueKey : uniqueKeys) { + List uniqueKeyColumns = + uniqueKey.getColumnNames(); + for (ConstraintKey.ConstraintKeyColumn uniqueKeyColumn : uniqueKeyColumns) { + Column column = table.columnWithName(uniqueKeyColumn.getColumnName()); + if (isEvenlySplitColumn(column)) { + return column; + } + } + } + } + + throw new UnsupportedOperationException( + String.format( + "Incremental snapshot for tables requires primary key/unique key," + + " but table %s doesn't have primary key.", + tableId)); + } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java index 05935d1701d9..04671d28f5b2 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java @@ -33,7 +33,6 @@ import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Column; -import io.debezium.relational.Table; import io.debezium.relational.TableId; import java.math.BigDecimal; @@ -67,8 +66,7 @@ public Collection generateSplits(TableId tableId) { LOG.info("Start splitting table {} into chunks...", tableId); long start = System.currentTimeMillis(); - Table table = dialect.queryTableSchema(jdbc, tableId).getTable(); - Column splitColumn = getSplitColumn(table); + Column splitColumn = getSplitColumn(jdbc, dialect, tableId); final List chunks; try { chunks = splitTableIntoChunks(jdbc, tableId, splitColumn); @@ -393,18 +391,4 @@ private static void maySleep(int count, TableId tableId) { LOG.info("JdbcSourceChunkSplitter has split {} chunks for table {}", count, tableId); } } - - public static Column getSplitColumn(Table table) { - List primaryKeys = table.primaryKeyColumns(); - if (primaryKeys.isEmpty()) { - throw new UnsupportedOperationException( - String.format( - "Incremental snapshot for tables requires primary key," - + " but table %s doesn't have primary key.", - table.id())); - } - - // use first field in primary key as the split key - return primaryKeys.get(0); - } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerDialect.java index 0494cd98e1c9..464d8637f76a 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerDialect.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerDialect.java @@ -19,7 +19,6 @@ import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; -import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; @@ -47,7 +46,7 @@ public class SqlServerDialect implements JdbcDataSourceDialect { private static final long serialVersionUID = 1L; - private final SourceConfig sourceConfig; + private final SqlServerSourceConfig sourceConfig; private transient SqlServerSchema sqlServerSchema; @@ -95,7 +94,7 @@ public List discoverDataCollections(JdbcSourceConfig sourceConfig) { @Override public TableChanges.TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) { if (sqlServerSchema == null) { - sqlServerSchema = new SqlServerSchema(); + sqlServerSchema = new SqlServerSchema(sourceConfig.getDbzConnectorConfig()); } return sqlServerSchema.getTableSchema(jdbc, tableId); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java index 3de596fd7dab..ac0b8165dbd4 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java @@ -30,7 +30,6 @@ import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Column; -import io.debezium.relational.Table; import io.debezium.relational.TableId; import lombok.extern.slf4j.Slf4j; @@ -64,8 +63,7 @@ public Collection generateSplits(TableId tableId) { log.info("Start splitting table {} into chunks...", tableId); long start = System.currentTimeMillis(); - Table table = dialect.queryTableSchema(jdbc, tableId).getTable(); - Column splitColumn = getSplitColumn(table); + Column splitColumn = getSplitColumn(jdbc, dialect, tableId); final List chunks; try { chunks = splitTableIntoChunks(jdbc, tableId, splitColumn); @@ -390,18 +388,4 @@ private static void maySleep(int count, TableId tableId) { log.info("JdbcSourceChunkSplitter has split {} chunks for table {}", count, tableId); } } - - public static Column getSplitColumn(Table table) { - List primaryKeys = table.primaryKeyColumns(); - if (primaryKeys.isEmpty()) { - throw new UnsupportedOperationException( - String.format( - "Incremental snapshot for tables requires primary key," - + " but table %s doesn't have primary key.", - table.id())); - } - - // use first field in primary key as the split key - return primaryKeys.get(0); - } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/utils/SqlServerSchema.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/utils/SqlServerSchema.java index 0e031a3cfd1f..83d51ae31ba9 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/utils/SqlServerSchema.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/utils/SqlServerSchema.java @@ -20,6 +20,7 @@ import org.apache.seatunnel.common.utils.SeaTunnelException; import io.debezium.connector.sqlserver.SqlServerConnection; +import io.debezium.connector.sqlserver.SqlServerConnectorConfig; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Table; import io.debezium.relational.TableId; @@ -29,18 +30,18 @@ import java.sql.SQLException; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; /** A component used to get schema by table path. */ public class SqlServerSchema { + private final SqlServerConnectorConfig connectorConfig; private final Map schemasByTableId; - public SqlServerSchema() { + public SqlServerSchema(SqlServerConnectorConfig connectorConfig) { this.schemasByTableId = new ConcurrentHashMap<>(); + this.connectorConfig = connectorConfig; } public TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) { @@ -55,16 +56,17 @@ public TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) { private TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) { SqlServerConnection sqlServerConnection = (SqlServerConnection) jdbc; - Set tableIdSet = new HashSet<>(); - tableIdSet.add(tableId); final Map tableChangeMap = new HashMap<>(); Tables tables = new Tables(); - tables.overwriteTable(tables.editOrCreateTable(tableId).create()); - try { sqlServerConnection.readSchema( - tables, tableId.catalog(), tableId.schema(), null, null, false); + tables, + tableId.catalog(), + tableId.schema(), + connectorConfig.getTableFilters().dataCollectionFilter(), + null, + false); Table table = tables.forTable(tableId); TableChange tableChange = new TableChange(TableChanges.TableChangeType.CREATE, table); tableChangeMap.put(tableId, tableChange); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java index 28da81432521..247ecc651fee 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java @@ -180,9 +180,12 @@ protected List getConstraintKeys( // index name -> index Map constraintKeyMap = new HashMap<>(); while (resultSet.next()) { - String indexName = resultSet.getString("INDEX_NAME"); String columnName = resultSet.getString("COLUMN_NAME"); - String unique = resultSet.getString("NON_UNIQUE"); + if (columnName == null) { + continue; + } + String indexName = resultSet.getString("INDEX_NAME"); + boolean noUnique = resultSet.getBoolean("NON_UNIQUE"); ConstraintKey constraintKey = constraintKeyMap.computeIfAbsent( @@ -190,8 +193,7 @@ protected List getConstraintKeys( s -> { ConstraintKey.ConstraintType constraintType = ConstraintKey.ConstraintType.KEY; - // 0 is unique. - if ("0".equals(unique)) { + if (!noUnique) { constraintType = ConstraintKey.ConstraintType.UNIQUE_KEY; } return ConstraintKey.of( diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java index a9bb1c15554b..ef3f985432ac 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.sink.DataSaveMode; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; @@ -37,6 +38,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.AUTO_COMMIT; import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.BATCH_INTERVAL_MS; @@ -78,6 +80,21 @@ public TableSink createSink(TableFactoryContext context) { PrimaryKey primaryKey = catalogTable.getTableSchema().getPrimaryKey(); if (primaryKey != null && !CollectionUtils.isEmpty(primaryKey.getColumnNames())) { map.put(PRIMARY_KEYS.key(), String.join(",", primaryKey.getColumnNames())); + } else { + Optional keyOptional = + catalogTable.getTableSchema().getConstraintKeys().stream() + .filter( + key -> + ConstraintKey.ConstraintType.UNIQUE_KEY.equals( + key.getConstraintType())) + .findFirst(); + if (keyOptional.isPresent()) { + map.put( + PRIMARY_KEYS.key(), + keyOptional.get().getColumnNames().stream() + .map(key -> key.getColumnName()) + .collect(Collectors.joining(","))); + } } config = ReadonlyConfig.fromMap(new HashMap<>(map)); } From e1be9d7f8a2c705712f9311a732560c1247de6d3 Mon Sep 17 00:00:00 2001 From: ic4y <83933160+ic4y@users.noreply.github.com> Date: Wed, 26 Jul 2023 21:09:35 +0800 Subject: [PATCH 20/43] [Feature][Connector-V2][CDC] Support string type shard fields. (#5147) * [feature][CDC base] Supports string type shard fields * Delete invalid code --- .../AbstractJdbcSourceChunkSplitter.java | 392 ++++++++++++++++++ .../splitter/JdbcSourceChunkSplitter.java | 45 +- .../cdc/base/utils/ObjectUtils.java | 2 + .../source/eumerator/MySqlChunkSplitter.java | 309 +------------- .../eumerator/SqlServerChunkSplitter.java | 304 +------------- 5 files changed, 401 insertions(+), 651 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitter.java diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitter.java new file mode 100644 index 000000000000..e956b111709b --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/AbstractJdbcSourceChunkSplitter.java @@ -0,0 +1,392 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter; + +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; +import org.apache.seatunnel.connectors.cdc.base.utils.ObjectUtils; + +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import java.math.BigDecimal; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +import static java.math.BigDecimal.ROUND_CEILING; +import static org.apache.seatunnel.connectors.cdc.base.utils.ObjectUtils.doubleCompare; + +@Slf4j +public abstract class AbstractJdbcSourceChunkSplitter implements JdbcSourceChunkSplitter { + + private final JdbcSourceConfig sourceConfig; + private final JdbcDataSourceDialect dialect; + + public AbstractJdbcSourceChunkSplitter( + JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dialect) { + this.sourceConfig = sourceConfig; + this.dialect = dialect; + } + + @Override + public Collection generateSplits(TableId tableId) { + try (JdbcConnection jdbc = dialect.openJdbcConnection(sourceConfig)) { + log.info("Start splitting table {} into chunks...", tableId); + long start = System.currentTimeMillis(); + + Column splitColumn = getSplitColumn(jdbc, dialect, tableId); + final List chunks; + try { + chunks = splitTableIntoChunks(jdbc, tableId, splitColumn); + } catch (SQLException e) { + throw new RuntimeException("Failed to split chunks for table " + tableId, e); + } + + // convert chunks into splits + List splits = new ArrayList<>(); + SeaTunnelRowType splitType = getSplitType(splitColumn); + for (int i = 0; i < chunks.size(); i++) { + ChunkRange chunk = chunks.get(i); + SnapshotSplit split = + createSnapshotSplit( + jdbc, + tableId, + i, + splitType, + chunk.getChunkStart(), + chunk.getChunkEnd()); + splits.add(split); + } + + long end = System.currentTimeMillis(); + log.info( + "Split table {} into {} chunks, time cost: {}ms.", + tableId, + splits.size(), + end - start); + return splits; + } catch (Exception e) { + throw new RuntimeException( + String.format("Generate Splits for table %s error", tableId), e); + } + } + + private List splitTableIntoChunks( + JdbcConnection jdbc, TableId tableId, Column splitColumn) throws SQLException { + final String splitColumnName = splitColumn.name(); + final Object[] minMax = queryMinMax(jdbc, tableId, splitColumnName); + final Object min = minMax[0]; + final Object max = minMax[1]; + if (min == null || max == null || min.equals(max)) { + // empty table, or only one row, return full table scan as a chunk + return Collections.singletonList(ChunkRange.all()); + } + + final int chunkSize = sourceConfig.getSplitSize(); + final double distributionFactorUpper = sourceConfig.getDistributionFactorUpper(); + final double distributionFactorLower = sourceConfig.getDistributionFactorLower(); + + if (isEvenlySplitColumn(splitColumn)) { + long approximateRowCnt = queryApproximateRowCnt(jdbc, tableId); + double distributionFactor = + calculateDistributionFactor(tableId, min, max, approximateRowCnt); + + boolean dataIsEvenlyDistributed = + doubleCompare(distributionFactor, distributionFactorLower) >= 0 + && doubleCompare(distributionFactor, distributionFactorUpper) <= 0; + + if (dataIsEvenlyDistributed) { + // the minimum dynamic chunk size is at least 1 + final int dynamicChunkSize = Math.max((int) (distributionFactor * chunkSize), 1); + return splitEvenlySizedChunks( + tableId, min, max, approximateRowCnt, chunkSize, dynamicChunkSize); + } else { + int shardCount = (int) (approximateRowCnt / chunkSize); + int inverseSamplingRate = sourceConfig.getInverseSamplingRate(); + if (sourceConfig.getSampleShardingThreshold() < shardCount) { + // It is necessary to ensure that the number of data rows sampled by the + // sampling rate is greater than the number of shards. + // Otherwise, if the sampling rate is too low, it may result in an insufficient + // number of data rows for the shards, leading to an inadequate number of + // shards. + // Therefore, inverseSamplingRate should be less than chunkSize + if (inverseSamplingRate > chunkSize) { + log.warn( + "The inverseSamplingRate is {}, which is greater than chunkSize {}, so we set inverseSamplingRate to chunkSize", + inverseSamplingRate, + chunkSize); + inverseSamplingRate = chunkSize; + } + Object[] sample = + sampleDataFromColumn( + jdbc, tableId, splitColumnName, inverseSamplingRate); + return efficientShardingThroughSampling( + tableId, sample, approximateRowCnt, shardCount); + } + return splitUnevenlySizedChunks( + jdbc, tableId, splitColumnName, min, max, chunkSize); + } + } else { + return splitUnevenlySizedChunks(jdbc, tableId, splitColumnName, min, max, chunkSize); + } + } + + /** Split table into unevenly sized chunks by continuously calculating next chunk max value. */ + protected List splitUnevenlySizedChunks( + JdbcConnection jdbc, + TableId tableId, + String splitColumnName, + Object min, + Object max, + int chunkSize) + throws SQLException { + log.info( + "Use unevenly-sized chunks for table {}, the chunk size is {}", tableId, chunkSize); + final List splits = new ArrayList<>(); + Object chunkStart = null; + Object chunkEnd = nextChunkEnd(jdbc, min, tableId, splitColumnName, max, chunkSize); + int count = 0; + while (chunkEnd != null && ObjectCompare(chunkEnd, max) <= 0) { + // we start from [null, min + chunk_size) and avoid [null, min) + splits.add(ChunkRange.of(chunkStart, chunkEnd)); + // may sleep a while to avoid DDOS on MySQL server + maySleep(count++, tableId); + chunkStart = chunkEnd; + chunkEnd = nextChunkEnd(jdbc, chunkEnd, tableId, splitColumnName, max, chunkSize); + } + // add the ending split + splits.add(ChunkRange.of(chunkStart, null)); + return splits; + } + + protected Object nextChunkEnd( + JdbcConnection jdbc, + Object previousChunkEnd, + TableId tableId, + String splitColumnName, + Object max, + int chunkSize) + throws SQLException { + // chunk end might be null when max values are removed + Object chunkEnd = + queryNextChunkMax(jdbc, tableId, splitColumnName, chunkSize, previousChunkEnd); + if (Objects.equals(previousChunkEnd, chunkEnd)) { + // we don't allow equal chunk start and end, + // should query the next one larger than chunkEnd + chunkEnd = queryMin(jdbc, tableId, splitColumnName, chunkEnd); + } + if (ObjectCompare(chunkEnd, max) >= 0) { + return null; + } else { + return chunkEnd; + } + } + + protected List efficientShardingThroughSampling( + TableId tableId, Object[] sampleData, long approximateRowCnt, int shardCount) { + log.info( + "Use efficient sharding through sampling optimization for table {}, the approximate row count is {}, the shardCount is {}", + tableId, + approximateRowCnt, + shardCount); + + final List splits = new ArrayList<>(); + + if (shardCount == 0) { + splits.add(ChunkRange.of(null, null)); + return splits; + } + + double approxSamplePerShard = (double) sampleData.length / shardCount; + + if (approxSamplePerShard <= 1) { + + splits.add(ChunkRange.of(null, sampleData[0])); + for (int i = 0; i < sampleData.length - 1; i++) { + splits.add(ChunkRange.of(sampleData[i], sampleData[i + 1])); + } + splits.add(ChunkRange.of(sampleData[sampleData.length - 1], null)); + } else { + // Calculate the shard boundaries + for (int i = 0; i < shardCount; i++) { + Object chunkStart = i == 0 ? null : sampleData[(int) (i * approxSamplePerShard)]; + Object chunkEnd = + i < shardCount - 1 + ? sampleData[(int) ((i + 1) * approxSamplePerShard)] + : null; + splits.add(ChunkRange.of(chunkStart, chunkEnd)); + } + } + return splits; + } + + /** + * Split table into evenly sized chunks based on the numeric min and max value of split column, + * and tumble chunks in step size. + */ + protected List splitEvenlySizedChunks( + TableId tableId, + Object min, + Object max, + long approximateRowCnt, + int chunkSize, + int dynamicChunkSize) { + log.info( + "Use evenly-sized chunk optimization for table {}, the approximate row count is {}, the chunk size is {}, the dynamic chunk size is {}", + tableId, + approximateRowCnt, + chunkSize, + dynamicChunkSize); + if (approximateRowCnt <= chunkSize) { + // there is no more than one chunk, return full table as a chunk + return Collections.singletonList(ChunkRange.all()); + } + + final List splits = new ArrayList<>(); + Object chunkStart = null; + Object chunkEnd = ObjectUtils.plus(min, dynamicChunkSize); + while (ObjectCompare(chunkEnd, max) <= 0) { + splits.add(ChunkRange.of(chunkStart, chunkEnd)); + chunkStart = chunkEnd; + try { + chunkEnd = ObjectUtils.plus(chunkEnd, dynamicChunkSize); + } catch (ArithmeticException e) { + // Stop chunk split to avoid dead loop when number overflows. + break; + } + } + // add the ending split + splits.add(ChunkRange.of(chunkStart, null)); + return splits; + } + + // ------------------------------------------------------------------------------------------ + /** Returns the distribution factor of the given table. */ + @SuppressWarnings("MagicNumber") + protected double calculateDistributionFactor( + TableId tableId, Object min, Object max, long approximateRowCnt) { + + if (!min.getClass().equals(max.getClass())) { + throw new IllegalStateException( + String.format( + "Unsupported operation type, the MIN value type %s is different with MAX value type %s.", + min.getClass().getSimpleName(), max.getClass().getSimpleName())); + } + if (approximateRowCnt == 0) { + return Double.MAX_VALUE; + } + BigDecimal difference = ObjectUtils.minus(max, min); + // factor = (max - min + 1) / rowCount + final BigDecimal subRowCnt = difference.add(BigDecimal.valueOf(1)); + double distributionFactor = + subRowCnt.divide(new BigDecimal(approximateRowCnt), 4, ROUND_CEILING).doubleValue(); + log.info( + "The distribution factor of table {} is {} according to the min split key {}, max split key {} and approximate row count {}", + tableId, + distributionFactor, + min, + max, + approximateRowCnt); + return distributionFactor; + } + + protected SnapshotSplit createSnapshotSplit( + JdbcConnection jdbc, + TableId tableId, + int chunkId, + SeaTunnelRowType splitKeyType, + Object chunkStart, + Object chunkEnd) { + // currently, we only support single split column + Object[] splitStart = chunkStart == null ? null : new Object[] {chunkStart}; + Object[] splitEnd = chunkEnd == null ? null : new Object[] {chunkEnd}; + return new SnapshotSplit( + splitId(tableId, chunkId), tableId, splitKeyType, splitStart, splitEnd); + } + + protected Column getSplitColumn( + JdbcConnection jdbc, JdbcDataSourceDialect dialect, TableId tableId) + throws SQLException { + Optional primaryKey = dialect.getPrimaryKey(jdbc, tableId); + if (primaryKey.isPresent()) { + List pkColumns = primaryKey.get().getColumnNames(); + + Table table = dialect.queryTableSchema(jdbc, tableId).getTable(); + for (String pkColumn : pkColumns) { + Column column = table.columnWithName(pkColumn); + if (isEvenlySplitColumn(column)) { + return column; + } + } + } + + List uniqueKeys = dialect.getUniqueKeys(jdbc, tableId); + if (!uniqueKeys.isEmpty()) { + Table table = dialect.queryTableSchema(jdbc, tableId).getTable(); + for (ConstraintKey uniqueKey : uniqueKeys) { + List uniqueKeyColumns = + uniqueKey.getColumnNames(); + for (ConstraintKey.ConstraintKeyColumn uniqueKeyColumn : uniqueKeyColumns) { + Column column = table.columnWithName(uniqueKeyColumn.getColumnName()); + if (isEvenlySplitColumn(column)) { + return column; + } + } + } + } + + throw new UnsupportedOperationException( + String.format( + "Incremental snapshot for tables requires primary key/unique key," + + " but table %s doesn't have primary key.", + tableId)); + } + + protected String splitId(TableId tableId, int chunkId) { + return tableId.toString() + ":" + chunkId; + } + + protected int ObjectCompare(Object obj1, Object obj2) { + return ObjectUtils.compare(obj1, obj2); + } + + @SuppressWarnings("MagicNumber") + private static void maySleep(int count, TableId tableId) { + // every 100 queries to sleep 1s + if (count % 10 == 0) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + // nothing to do + } + log.info("JdbcSourceChunkSplitter has split {} chunks for table {}", count, tableId); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java index bbad9d04b1e1..b271be0d7653 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/enumerator/splitter/JdbcSourceChunkSplitter.java @@ -17,22 +17,16 @@ package org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter; -import org.apache.seatunnel.api.table.catalog.ConstraintKey; -import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Column; -import io.debezium.relational.Table; import io.debezium.relational.TableId; import java.sql.SQLException; import java.util.Collection; -import java.util.List; -import java.util.Optional; /** The {@code ChunkSplitter} used to split table into a set of chunks for JDBC data source. */ public interface JdbcSourceChunkSplitter extends ChunkSplitter { @@ -142,6 +136,7 @@ default boolean isEvenlySplitColumn(Column splitColumn) { case INT: case BIGINT: case DECIMAL: + case STRING: return true; default: return false; @@ -167,42 +162,4 @@ default SeaTunnelRowType getSplitType(Column splitColumn) { new String[] {splitColumn.name()}, new SeaTunnelDataType[] {fromDbzColumn(splitColumn)}); } - - default Column getSplitColumn( - JdbcConnection jdbc, JdbcDataSourceDialect dialect, TableId tableId) - throws SQLException { - Optional primaryKey = dialect.getPrimaryKey(jdbc, tableId); - if (primaryKey.isPresent()) { - List pkColumns = primaryKey.get().getColumnNames(); - - Table table = dialect.queryTableSchema(jdbc, tableId).getTable(); - for (String pkColumn : pkColumns) { - Column column = table.columnWithName(pkColumn); - if (isEvenlySplitColumn(column)) { - return column; - } - } - } - - List uniqueKeys = dialect.getUniqueKeys(jdbc, tableId); - if (!uniqueKeys.isEmpty()) { - Table table = dialect.queryTableSchema(jdbc, tableId).getTable(); - for (ConstraintKey uniqueKey : uniqueKeys) { - List uniqueKeyColumns = - uniqueKey.getColumnNames(); - for (ConstraintKey.ConstraintKeyColumn uniqueKeyColumn : uniqueKeyColumns) { - Column column = table.columnWithName(uniqueKeyColumn.getColumnName()); - if (isEvenlySplitColumn(column)) { - return column; - } - } - } - } - - throw new UnsupportedOperationException( - String.format( - "Incremental snapshot for tables requires primary key/unique key," - + " but table %s doesn't have primary key.", - tableId)); - } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/ObjectUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/ObjectUtils.java index 3c5b669a257a..0f703f02c1cb 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/ObjectUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/utils/ObjectUtils.java @@ -63,6 +63,8 @@ public static BigDecimal minus(Object minuend, Object subtrahend) { ((BigInteger) minuend).subtract((BigInteger) subtrahend).toString()); } else if (minuend instanceof BigDecimal) { return ((BigDecimal) minuend).subtract((BigDecimal) subtrahend); + } else if (minuend instanceof String) { + return BigDecimal.valueOf(Long.MAX_VALUE); } else { throw new UnsupportedOperationException( String.format( diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java index 04671d28f5b2..0249889b239c 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/eumerator/MySqlChunkSplitter.java @@ -21,86 +21,21 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; -import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkRange; -import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.JdbcSourceChunkSplitter; -import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; -import org.apache.seatunnel.connectors.cdc.base.utils.ObjectUtils; +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.AbstractJdbcSourceChunkSplitter; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlTypeUtils; import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.utils.MySqlUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Column; import io.debezium.relational.TableId; -import java.math.BigDecimal; import java.sql.SQLException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -import static java.math.BigDecimal.ROUND_CEILING; -import static org.apache.seatunnel.connectors.cdc.base.utils.ObjectUtils.doubleCompare; /** The {@code ChunkSplitter} used to split table into a set of chunks for JDBC data source. */ -public class MySqlChunkSplitter implements JdbcSourceChunkSplitter { - - private static final Logger LOG = LoggerFactory.getLogger(MySqlChunkSplitter.class); - - private final JdbcSourceConfig sourceConfig; - private final JdbcDataSourceDialect dialect; +public class MySqlChunkSplitter extends AbstractJdbcSourceChunkSplitter { public MySqlChunkSplitter(JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dialect) { - this.sourceConfig = sourceConfig; - this.dialect = dialect; - } - - @Override - public Collection generateSplits(TableId tableId) { - try (JdbcConnection jdbc = dialect.openJdbcConnection(sourceConfig)) { - - LOG.info("Start splitting table {} into chunks...", tableId); - long start = System.currentTimeMillis(); - - Column splitColumn = getSplitColumn(jdbc, dialect, tableId); - final List chunks; - try { - chunks = splitTableIntoChunks(jdbc, tableId, splitColumn); - } catch (SQLException e) { - throw new RuntimeException("Failed to split chunks for table " + tableId, e); - } - - // convert chunks into splits - List splits = new ArrayList<>(); - SeaTunnelRowType splitType = getSplitType(splitColumn); - for (int i = 0; i < chunks.size(); i++) { - ChunkRange chunk = chunks.get(i); - SnapshotSplit split = - createSnapshotSplit( - jdbc, - tableId, - i, - splitType, - chunk.getChunkStart(), - chunk.getChunkEnd()); - splits.add(split); - } - - long end = System.currentTimeMillis(); - LOG.info( - "Split table {} into {} chunks, time cost: {}ms.", - tableId, - splits.size(), - end - start); - return splits; - } catch (Exception e) { - throw new RuntimeException( - String.format("Generate Splits for table %s error", tableId), e); - } + super(sourceConfig, dialect); } @Override @@ -153,242 +88,4 @@ public String buildSplitScanQuery( public SeaTunnelDataType fromDbzColumn(Column splitColumn) { return MySqlTypeUtils.convertFromColumn(splitColumn); } - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - /** - * We can use evenly-sized chunks or unevenly-sized chunks when split table into chunks, using - * evenly-sized chunks which is much efficient, using unevenly-sized chunks which will request - * many queries and is not efficient. - */ - private List splitTableIntoChunks( - JdbcConnection jdbc, TableId tableId, Column splitColumn) throws SQLException { - final String splitColumnName = splitColumn.name(); - final Object[] minMax = queryMinMax(jdbc, tableId, splitColumnName); - final Object min = minMax[0]; - final Object max = minMax[1]; - if (min == null || max == null || min.equals(max)) { - // empty table, or only one row, return full table scan as a chunk - return Collections.singletonList(ChunkRange.all()); - } - - final int chunkSize = sourceConfig.getSplitSize(); - final double distributionFactorUpper = sourceConfig.getDistributionFactorUpper(); - final double distributionFactorLower = sourceConfig.getDistributionFactorLower(); - - if (isEvenlySplitColumn(splitColumn)) { - long approximateRowCnt = queryApproximateRowCnt(jdbc, tableId); - double distributionFactor = - calculateDistributionFactor(tableId, min, max, approximateRowCnt); - - boolean dataIsEvenlyDistributed = - doubleCompare(distributionFactor, distributionFactorLower) >= 0 - && doubleCompare(distributionFactor, distributionFactorUpper) <= 0; - - if (dataIsEvenlyDistributed) { - // the minimum dynamic chunk size is at least 1 - final int dynamicChunkSize = Math.max((int) (distributionFactor * chunkSize), 1); - return splitEvenlySizedChunks( - tableId, min, max, approximateRowCnt, chunkSize, dynamicChunkSize); - } else { - int shardCount = (int) (approximateRowCnt / chunkSize); - if (sourceConfig.getSampleShardingThreshold() < shardCount) { - Object[] sample = - sampleDataFromColumn( - jdbc, - tableId, - splitColumnName, - sourceConfig.getInverseSamplingRate()); - // In order to prevent data loss due to the absence of the minimum value in the - // sampled data, the minimum value is directly added here. - Object[] newSample = new Object[sample.length + 1]; - newSample[0] = min; - System.arraycopy(sample, 0, newSample, 1, sample.length); - return efficientShardingThroughSampling( - tableId, newSample, approximateRowCnt, shardCount); - } - return splitUnevenlySizedChunks( - jdbc, tableId, splitColumnName, min, max, chunkSize); - } - } else { - return splitUnevenlySizedChunks(jdbc, tableId, splitColumnName, min, max, chunkSize); - } - } - - private List efficientShardingThroughSampling( - TableId tableId, Object[] sampleData, long approximateRowCnt, int shardCount) { - LOG.info( - "Use efficient sharding through sampling optimization for table {}, the approximate row count is {}, the shardCount is {}", - tableId, - approximateRowCnt, - shardCount); - - final List splits = new ArrayList<>(); - - // Calculate the shard boundaries - for (int i = 0; i < shardCount; i++) { - Object chunkStart = sampleData[(int) ((long) i * sampleData.length / shardCount)]; - Object chunkEnd = - i < shardCount - 1 - ? sampleData[(int) (((long) i + 1) * sampleData.length / shardCount)] - : null; - splits.add(ChunkRange.of(chunkStart, chunkEnd)); - } - - return splits; - } - - /** - * Split table into evenly sized chunks based on the numeric min and max value of split column, - * and tumble chunks in step size. - */ - private List splitEvenlySizedChunks( - TableId tableId, - Object min, - Object max, - long approximateRowCnt, - int chunkSize, - int dynamicChunkSize) { - LOG.info( - "Use evenly-sized chunk optimization for table {}, the approximate row count is {}, the chunk size is {}, the dynamic chunk size is {}", - tableId, - approximateRowCnt, - chunkSize, - dynamicChunkSize); - if (approximateRowCnt <= chunkSize) { - // there is no more than one chunk, return full table as a chunk - return Collections.singletonList(ChunkRange.all()); - } - - final List splits = new ArrayList<>(); - Object chunkStart = null; - Object chunkEnd = ObjectUtils.plus(min, dynamicChunkSize); - while (ObjectUtils.compare(chunkEnd, max) <= 0) { - splits.add(ChunkRange.of(chunkStart, chunkEnd)); - chunkStart = chunkEnd; - try { - chunkEnd = ObjectUtils.plus(chunkEnd, dynamicChunkSize); - } catch (ArithmeticException e) { - // Stop chunk split to avoid dead loop when number overflows. - break; - } - } - // add the ending split - splits.add(ChunkRange.of(chunkStart, null)); - return splits; - } - - /** Split table into unevenly sized chunks by continuously calculating next chunk max value. */ - private List splitUnevenlySizedChunks( - JdbcConnection jdbc, - TableId tableId, - String splitColumnName, - Object min, - Object max, - int chunkSize) - throws SQLException { - LOG.info( - "Use unevenly-sized chunks for table {}, the chunk size is {}", tableId, chunkSize); - final List splits = new ArrayList<>(); - Object chunkStart = null; - Object chunkEnd = nextChunkEnd(jdbc, min, tableId, splitColumnName, max, chunkSize); - int count = 0; - while (chunkEnd != null && ObjectUtils.compare(chunkEnd, max) <= 0) { - // we start from [null, min + chunk_size) and avoid [null, min) - splits.add(ChunkRange.of(chunkStart, chunkEnd)); - // may sleep a while to avoid DDOS on MySQL server - maySleep(count++, tableId); - chunkStart = chunkEnd; - chunkEnd = nextChunkEnd(jdbc, chunkEnd, tableId, splitColumnName, max, chunkSize); - } - // add the ending split - splits.add(ChunkRange.of(chunkStart, null)); - return splits; - } - - private Object nextChunkEnd( - JdbcConnection jdbc, - Object previousChunkEnd, - TableId tableId, - String splitColumnName, - Object max, - int chunkSize) - throws SQLException { - // chunk end might be null when max values are removed - Object chunkEnd = - queryNextChunkMax(jdbc, tableId, splitColumnName, chunkSize, previousChunkEnd); - if (Objects.equals(previousChunkEnd, chunkEnd)) { - // we don't allow equal chunk start and end, - // should query the next one larger than chunkEnd - chunkEnd = queryMin(jdbc, tableId, splitColumnName, chunkEnd); - } - if (ObjectUtils.compare(chunkEnd, max) >= 0) { - return null; - } else { - return chunkEnd; - } - } - - private SnapshotSplit createSnapshotSplit( - JdbcConnection jdbc, - TableId tableId, - int chunkId, - SeaTunnelRowType splitKeyType, - Object chunkStart, - Object chunkEnd) { - // currently, we only support single split column - Object[] splitStart = chunkStart == null ? null : new Object[] {chunkStart}; - Object[] splitEnd = chunkEnd == null ? null : new Object[] {chunkEnd}; - return new SnapshotSplit( - splitId(tableId, chunkId), tableId, splitKeyType, splitStart, splitEnd); - } - - // ------------------------------------------------------------------------------------------ - /** Returns the distribution factor of the given table. */ - @SuppressWarnings("MagicNumber") - private double calculateDistributionFactor( - TableId tableId, Object min, Object max, long approximateRowCnt) { - - if (!min.getClass().equals(max.getClass())) { - throw new IllegalStateException( - String.format( - "Unsupported operation type, the MIN value type %s is different with MAX value type %s.", - min.getClass().getSimpleName(), max.getClass().getSimpleName())); - } - if (approximateRowCnt == 0) { - return Double.MAX_VALUE; - } - BigDecimal difference = ObjectUtils.minus(max, min); - // factor = (max - min + 1) / rowCount - final BigDecimal subRowCnt = difference.add(BigDecimal.valueOf(1)); - double distributionFactor = - subRowCnt.divide(new BigDecimal(approximateRowCnt), 4, ROUND_CEILING).doubleValue(); - LOG.info( - "The distribution factor of table {} is {} according to the min split key {}, max split key {} and approximate row count {}", - tableId, - distributionFactor, - min, - max, - approximateRowCnt); - return distributionFactor; - } - - private static String splitId(TableId tableId, int chunkId) { - return tableId.toString() + ":" + chunkId; - } - - @SuppressWarnings("MagicNumber") - private static void maySleep(int count, TableId tableId) { - // every 100 queries to sleep 1s - if (count % 10 == 0) { - try { - Thread.sleep(100); - } catch (InterruptedException e) { - // nothing to do - } - LOG.info("JdbcSourceChunkSplitter has split {} chunks for table {}", count, tableId); - } - } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java index ac0b8165dbd4..7efd53dc3fc2 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/eumerator/SqlServerChunkSplitter.java @@ -21,10 +21,7 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; -import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkRange; -import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.JdbcSourceChunkSplitter; -import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; -import org.apache.seatunnel.connectors.cdc.base.utils.ObjectUtils; +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.AbstractJdbcSourceChunkSplitter; import org.apache.seatunnel.connectors.seatunnel.cdc.sqlserver.source.utils.SqlServerTypeUtils; import org.apache.seatunnel.connectors.seatunnel.cdc.sqlserver.source.utils.SqlServerUtils; @@ -33,71 +30,14 @@ import io.debezium.relational.TableId; import lombok.extern.slf4j.Slf4j; -import java.math.BigDecimal; import java.sql.SQLException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - -import static java.math.BigDecimal.ROUND_CEILING; -import static org.apache.seatunnel.connectors.cdc.base.utils.ObjectUtils.doubleCompare; /** The {@code ChunkSplitter} used to split table into a set of chunks for JDBC data source. */ @Slf4j -public class SqlServerChunkSplitter implements JdbcSourceChunkSplitter { - - private final JdbcSourceConfig sourceConfig; - private final JdbcDataSourceDialect dialect; +public class SqlServerChunkSplitter extends AbstractJdbcSourceChunkSplitter { public SqlServerChunkSplitter(JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dialect) { - this.sourceConfig = sourceConfig; - this.dialect = dialect; - } - - @Override - public Collection generateSplits(TableId tableId) { - try (JdbcConnection jdbc = dialect.openJdbcConnection(sourceConfig)) { - - log.info("Start splitting table {} into chunks...", tableId); - long start = System.currentTimeMillis(); - - Column splitColumn = getSplitColumn(jdbc, dialect, tableId); - final List chunks; - try { - chunks = splitTableIntoChunks(jdbc, tableId, splitColumn); - } catch (SQLException e) { - throw new RuntimeException("Failed to split chunks for table " + tableId, e); - } - - // convert chunks into splits - List splits = new ArrayList<>(); - SeaTunnelRowType splitType = getSplitType(splitColumn); - for (int i = 0; i < chunks.size(); i++) { - ChunkRange chunk = chunks.get(i); - SnapshotSplit split = - createSnapshotSplit( - jdbc, - tableId, - i, - splitType, - chunk.getChunkStart(), - chunk.getChunkEnd()); - splits.add(split); - } - - long end = System.currentTimeMillis(); - log.info( - "Split table {} into {} chunks, time cost: {}ms.", - tableId, - splits.size(), - end - start); - return splits; - } catch (Exception e) { - throw new RuntimeException( - String.format("Generate Splits for table %s error", tableId), e); - } + super(sourceConfig, dialect); } @Override @@ -150,242 +90,4 @@ public String buildSplitScanQuery( public SeaTunnelDataType fromDbzColumn(Column splitColumn) { return SqlServerTypeUtils.convertFromColumn(splitColumn); } - - // -------------------------------------------------------------------------------------------- - // Utilities - // -------------------------------------------------------------------------------------------- - - /** - * We can use evenly-sized chunks or unevenly-sized chunks when split table into chunks, using - * evenly-sized chunks which is much efficient, using unevenly-sized chunks which will request - * many queries and is not efficient. - */ - private List splitTableIntoChunks( - JdbcConnection jdbc, TableId tableId, Column splitColumn) throws SQLException { - final String splitColumnName = splitColumn.name(); - final Object[] minMax = queryMinMax(jdbc, tableId, splitColumnName); - final Object min = minMax[0]; - final Object max = minMax[1]; - if (min == null || max == null || min.equals(max)) { - // empty table, or only one row, return full table scan as a chunk - return Collections.singletonList(ChunkRange.all()); - } - - final int chunkSize = sourceConfig.getSplitSize(); - final double distributionFactorUpper = sourceConfig.getDistributionFactorUpper(); - final double distributionFactorLower = sourceConfig.getDistributionFactorLower(); - - if (isEvenlySplitColumn(splitColumn)) { - long approximateRowCnt = queryApproximateRowCnt(jdbc, tableId); - double distributionFactor = - calculateDistributionFactor(tableId, min, max, approximateRowCnt); - - boolean dataIsEvenlyDistributed = - doubleCompare(distributionFactor, distributionFactorLower) >= 0 - && doubleCompare(distributionFactor, distributionFactorUpper) <= 0; - - if (dataIsEvenlyDistributed) { - // the minimum dynamic chunk size is at least 1 - final int dynamicChunkSize = Math.max((int) (distributionFactor * chunkSize), 1); - return splitEvenlySizedChunks( - tableId, min, max, approximateRowCnt, chunkSize, dynamicChunkSize); - } else { - int shardCount = (int) (approximateRowCnt / chunkSize); - if (sourceConfig.getSampleShardingThreshold() < shardCount) { - Object[] sample = - sampleDataFromColumn( - jdbc, - tableId, - splitColumnName, - sourceConfig.getInverseSamplingRate()); - // In order to prevent data loss due to the absence of the minimum value in the - // sampled data, the minimum value is directly added here. - Object[] newSample = new Object[sample.length + 1]; - newSample[0] = min; - System.arraycopy(sample, 0, newSample, 1, sample.length); - return efficientShardingThroughSampling( - tableId, newSample, approximateRowCnt, shardCount); - } - return splitUnevenlySizedChunks( - jdbc, tableId, splitColumnName, min, max, chunkSize); - } - } else { - return splitUnevenlySizedChunks(jdbc, tableId, splitColumnName, min, max, chunkSize); - } - } - - private List efficientShardingThroughSampling( - TableId tableId, Object[] sampleData, long approximateRowCnt, int shardCount) { - log.info( - "Use efficient sharding through sampling optimization for table {}, the approximate row count is {}, the shardCount is {}", - tableId, - approximateRowCnt, - shardCount); - - final List splits = new ArrayList<>(); - - // Calculate the shard boundaries - for (int i = 0; i < shardCount; i++) { - Object chunkStart = sampleData[(int) ((long) i * sampleData.length / shardCount)]; - Object chunkEnd = - i < shardCount - 1 - ? sampleData[(int) (((long) i + 1) * sampleData.length / shardCount)] - : null; - splits.add(ChunkRange.of(chunkStart, chunkEnd)); - } - - return splits; - } - - /** - * Split table into evenly sized chunks based on the numeric min and max value of split column, - * and tumble chunks in step size. - */ - private List splitEvenlySizedChunks( - TableId tableId, - Object min, - Object max, - long approximateRowCnt, - int chunkSize, - int dynamicChunkSize) { - log.info( - "Use evenly-sized chunk optimization for table {}, the approximate row count is {}, the chunk size is {}, the dynamic chunk size is {}", - tableId, - approximateRowCnt, - chunkSize, - dynamicChunkSize); - if (approximateRowCnt <= chunkSize) { - // there is no more than one chunk, return full table as a chunk - return Collections.singletonList(ChunkRange.all()); - } - - final List splits = new ArrayList<>(); - Object chunkStart = null; - Object chunkEnd = ObjectUtils.plus(min, dynamicChunkSize); - while (ObjectUtils.compare(chunkEnd, max) <= 0) { - splits.add(ChunkRange.of(chunkStart, chunkEnd)); - chunkStart = chunkEnd; - try { - chunkEnd = ObjectUtils.plus(chunkEnd, dynamicChunkSize); - } catch (ArithmeticException e) { - // Stop chunk split to avoid dead loop when number overflows. - break; - } - } - // add the ending split - splits.add(ChunkRange.of(chunkStart, null)); - return splits; - } - - /** Split table into unevenly sized chunks by continuously calculating next chunk max value. */ - private List splitUnevenlySizedChunks( - JdbcConnection jdbc, - TableId tableId, - String splitColumnName, - Object min, - Object max, - int chunkSize) - throws SQLException { - log.info( - "Use unevenly-sized chunks for table {}, the chunk size is {}", tableId, chunkSize); - final List splits = new ArrayList<>(); - Object chunkStart = null; - Object chunkEnd = nextChunkEnd(jdbc, min, tableId, splitColumnName, max, chunkSize); - int count = 0; - while (chunkEnd != null && ObjectUtils.compare(chunkEnd, max) <= 0) { - // we start from [null, min + chunk_size) and avoid [null, min) - splits.add(ChunkRange.of(chunkStart, chunkEnd)); - // may sleep a while to avoid DDOS on MySQL server - maySleep(count++, tableId); - chunkStart = chunkEnd; - chunkEnd = nextChunkEnd(jdbc, chunkEnd, tableId, splitColumnName, max, chunkSize); - } - // add the ending split - splits.add(ChunkRange.of(chunkStart, null)); - return splits; - } - - private Object nextChunkEnd( - JdbcConnection jdbc, - Object previousChunkEnd, - TableId tableId, - String splitColumnName, - Object max, - int chunkSize) - throws SQLException { - // chunk end might be null when max values are removed - Object chunkEnd = - queryNextChunkMax(jdbc, tableId, splitColumnName, chunkSize, previousChunkEnd); - if (Objects.equals(previousChunkEnd, chunkEnd)) { - // we don't allow equal chunk start and end, - // should query the next one larger than chunkEnd - chunkEnd = queryMin(jdbc, tableId, splitColumnName, chunkEnd); - } - if (ObjectUtils.compare(chunkEnd, max) >= 0) { - return null; - } else { - return chunkEnd; - } - } - - private SnapshotSplit createSnapshotSplit( - JdbcConnection jdbc, - TableId tableId, - int chunkId, - SeaTunnelRowType splitKeyType, - Object chunkStart, - Object chunkEnd) { - // currently, we only support single split column - Object[] splitStart = chunkStart == null ? null : new Object[] {chunkStart}; - Object[] splitEnd = chunkEnd == null ? null : new Object[] {chunkEnd}; - return new SnapshotSplit( - splitId(tableId, chunkId), tableId, splitKeyType, splitStart, splitEnd); - } - - // ------------------------------------------------------------------------------------------ - /** Returns the distribution factor of the given table. */ - @SuppressWarnings("MagicNumber") - private double calculateDistributionFactor( - TableId tableId, Object min, Object max, long approximateRowCnt) { - - if (!min.getClass().equals(max.getClass())) { - throw new IllegalStateException( - String.format( - "Unsupported operation type, the MIN value type %s is different with MAX value type %s.", - min.getClass().getSimpleName(), max.getClass().getSimpleName())); - } - if (approximateRowCnt == 0) { - return Double.MAX_VALUE; - } - BigDecimal difference = ObjectUtils.minus(max, min); - // factor = (max - min + 1) / rowCount - final BigDecimal subRowCnt = difference.add(BigDecimal.valueOf(1)); - double distributionFactor = - subRowCnt.divide(new BigDecimal(approximateRowCnt), 4, ROUND_CEILING).doubleValue(); - log.info( - "The distribution factor of table {} is {} according to the min split key {}, max split key {} and approximate row count {}", - tableId, - distributionFactor, - min, - max, - approximateRowCnt); - return distributionFactor; - } - - private static String splitId(TableId tableId, int chunkId) { - return tableId.toString() + ":" + chunkId; - } - - @SuppressWarnings("MagicNumber") - private static void maySleep(int count, TableId tableId) { - // every 100 queries to sleep 1s - if (count % 10 == 0) { - try { - Thread.sleep(100); - } catch (InterruptedException e) { - // nothing to do - } - log.info("JdbcSourceChunkSplitter has split {} chunks for table {}", count, tableId); - } - } } From 1f94676436b730f6ef7b4d1e5d18fc2d689495e3 Mon Sep 17 00:00:00 2001 From: dengdi <114273849+dengd1937@users.noreply.github.com> Date: Wed, 26 Jul 2023 23:43:09 +0800 Subject: [PATCH 21/43] [Feature][Connector-V2][File] Add cos source&sink (#4979) * [Feature][Connector-V2][File] Add cos sink * update doc&e2e and add pom file header * add e2e file header and config * add file-cos module into dist pom.xml * [Feature][Connector-V2][File] Add cos source --------- Co-authored-by: dengd1937 --- docs/en/connector-v2/sink/CosFile.md | 259 ++++++++++++++++ docs/en/connector-v2/source/CosFile.md | 289 ++++++++++++++++++ plugin-mapping.properties | 2 + .../seatunnel/file/config/FileSystemType.java | 1 + .../connector-file/connector-file-cos/pom.xml | 64 ++++ .../seatunnel/file/cos/config/CosConf.java | 59 ++++ .../seatunnel/file/cos/config/CosConfig.java | 39 +++ .../seatunnel/file/cos/sink/CosFileSink.java | 63 ++++ .../file/cos/sink/CosFileSinkFactory.java | 88 ++++++ .../file/cos/source/CosFileSource.java | 119 ++++++++ .../file/cos/source/CosFileSourceFactory.java | 70 +++++ .../services/org.apache.hadoop.fs.FileSystem | 16 + .../file/cos/CosFileFactoryTest.java | 33 ++ .../connector-file/pom.xml | 1 + seatunnel-dist/pom.xml | 6 + .../connector-file-cos-e2e/pom.xml | 48 +++ .../e2e/connector/file/cos/CosFileIT.java | 76 +++++ .../resources/excel/cos_excel_to_assert.conf | 116 +++++++ .../resources/excel/fake_to_cos_excel.conf | 82 +++++ .../json/cos_file_json_to_assert.conf | 114 +++++++ .../resources/json/fake_to_cos_file_json.conf | 83 +++++ .../resources/orc/cos_file_orc_to_assert.conf | 80 +++++ .../resources/orc/fake_to_cos_file_orc.conf | 84 +++++ .../parquet/cos_file_parquet_to_assert.conf | 80 +++++ .../parquet/fake_to_cos_file_parquet.conf | 84 +++++ .../text/cos_file_text_to_assert.conf | 114 +++++++ .../resources/text/fake_to_cos_file_text.conf | 84 +++++ .../seatunnel-connector-v2-e2e/pom.xml | 1 + 28 files changed, 2155 insertions(+) create mode 100644 docs/en/connector-v2/sink/CosFile.md create mode 100644 docs/en/connector-v2/source/CosFile.md create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-cos/pom.xml create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/config/CosConf.java create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/config/CosConfig.java create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/sink/CosFileSink.java create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/sink/CosFileSinkFactory.java create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSource.java create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/resources/services/org.apache.hadoop.fs.FileSystem create mode 100644 seatunnel-connectors-v2/connector-file/connector-file-cos/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/cos/CosFileFactoryTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/cos/CosFileIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/excel/cos_excel_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/excel/fake_to_cos_excel.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/json/cos_file_json_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/json/fake_to_cos_file_json.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/orc/cos_file_orc_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/orc/fake_to_cos_file_orc.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/parquet/cos_file_parquet_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/parquet/fake_to_cos_file_parquet.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/text/cos_file_text_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/text/fake_to_cos_file_text.conf diff --git a/docs/en/connector-v2/sink/CosFile.md b/docs/en/connector-v2/sink/CosFile.md new file mode 100644 index 000000000000..563b174c3c82 --- /dev/null +++ b/docs/en/connector-v2/sink/CosFile.md @@ -0,0 +1,259 @@ +# CosFile + +> Cos file sink connector + +## Description + +Output data to cos file system. + +:::tip + +If you use spark/flink, In order to use this connector, You must ensure your spark/flink cluster already integrated hadoop. The tested hadoop version is 2.x. + +If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you download and install SeaTunnel Engine. You can check the jar package under ${SEATUNNEL_HOME}/lib to confirm this. + +To use this connector you need put hadoop-cos-{hadoop.version}-{version}.jar and cos_api-bundle-{version}.jar in ${SEATUNNEL_HOME}/lib dir, download: [Hadoop-Cos-release](https://github.com/tencentyun/hadoop-cos/releases). It only supports hadoop version 2.6.5+ and version 8.0.2+. + +::: + +## Key features + +- [x] [exactly-once](../../concept/connector-v2-features.md) + +By default, we use 2PC commit to ensure `exactly-once` + +- [x] file format type + - [x] text + - [x] csv + - [x] parquet + - [x] orc + - [x] json + - [x] excel + +## Options + +| name | type | required | default value | remarks | +|----------------------------------|---------|----------|--------------------------------------------|-----------------------------------------------------------| +| path | string | yes | - | | +| bucket | string | yes | - | | +| secret_id | string | yes | - | | +| secret_key | string | yes | - | | +| region | string | yes | - | | +| custom_filename | boolean | no | false | Whether you need custom the filename | +| file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | +| filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | +| file_format_type | string | no | "csv" | | +| field_delimiter | string | no | '\001' | Only used when file_format is text | +| row_delimiter | string | no | "\n" | Only used when file_format is text | +| have_partition | boolean | no | false | Whether you need processing partitions. | +| partition_by | array | no | - | Only used then have_partition is true | +| partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | +| is_partition_field_write_in_file | boolean | no | false | Only used then have_partition is true | +| sink_columns | array | no | | When this parameter is empty, all fields are sink columns | +| is_enable_transaction | boolean | no | true | | +| batch_size | int | no | 1000000 | | +| compress_codec | string | no | none | | +| common-options | object | no | - | | +| max_rows_in_memory | int | no | - | Only used when file_format is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | + +### path [string] + +The target dir path is required. + +### bucket [string] + +The bucket address of cos file system, for example: `cosn://seatunnel-test-1259587829` + +### secret_id [string] + +The secret id of cos file system. + +### secret_key [string] + +The secret key of cos file system. + +### region [string] + +The region of cos file system. + +### custom_filename [boolean] + +Whether custom the filename + +### file_name_expression [string] + +Only used when `custom_filename` is `true` + +`file_name_expression` describes the file expression which will be created into the `path`. We can add the variable `${now}` or `${uuid}` in the `file_name_expression`, like `test_${uuid}_${now}`, +`${now}` represents the current time, and its format can be defined by specifying the option `filename_time_format`. + +Please note that, If `is_enable_transaction` is `true`, we will auto add `${transactionId}_` in the head of the file. + +### filename_time_format [string] + +Only used when `custom_filename` is `true` + +When the format in the `file_name_expression` parameter is `xxxx-${now}` , `filename_time_format` can specify the time format of the path, and the default value is `yyyy.MM.dd` . The commonly used time formats are listed as follows: + +| Symbol | Description | +|--------|--------------------| +| y | Year | +| M | Month | +| d | Day of month | +| H | Hour in day (0-23) | +| m | Minute in hour | +| s | Second in minute | + +### file_format_type [string] + +We supported as the following file types: + +`text` `json` `csv` `orc` `parquet` `excel` + +Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. + +### field_delimiter [string] + +The separator between columns in a row of data. Only needed by `text` file format. + +### row_delimiter [string] + +The separator between rows in a file. Only needed by `text` file format. + +### have_partition [boolean] + +Whether you need processing partitions. + +### partition_by [array] + +Only used when `have_partition` is `true`. + +Partition data based on selected fields. + +### partition_dir_expression [string] + +Only used when `have_partition` is `true`. + +If the `partition_by` is specified, we will generate the corresponding partition directory based on the partition information, and the final file will be placed in the partition directory. + +Default `partition_dir_expression` is `${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/`. `k0` is the first partition field and `v0` is the value of the first partition field. + +### is_partition_field_write_in_file [boolean] + +Only used when `have_partition` is `true`. + +If `is_partition_field_write_in_file` is `true`, the partition field and the value of it will be write into data file. + +For example, if you want to write a Hive Data File, Its value should be `false`. + +### sink_columns [array] + +Which columns need be written to file, default value is all the columns get from `Transform` or `Source`. +The order of the fields determines the order in which the file is actually written. + +### is_enable_transaction [boolean] + +If `is_enable_transaction` is true, we will ensure that data will not be lost or duplicated when it is written to the target directory. + +Please note that, If `is_enable_transaction` is `true`, we will auto add `${transactionId}_` in the head of the file. + +Only support `true` now. + +### batch_size [int] + +The maximum number of rows in a file. For SeaTunnel Engine, the number of lines in the file is determined by `batch_size` and `checkpoint.interval` jointly decide. If the value of `checkpoint.interval` is large enough, sink writer will write rows in a file until the rows in the file larger than `batch_size`. If `checkpoint.interval` is small, the sink writer will create a new file when a new checkpoint trigger. + +### compress_codec [string] + +The compress codec of files and the details that supported as the following shown: + +- txt: `lzo` `none` +- json: `lzo` `none` +- csv: `lzo` `none` +- orc: `lzo` `snappy` `lz4` `zlib` `none` +- parquet: `lzo` `snappy` `lz4` `gzip` `brotli` `zstd` `none` + +Tips: excel type does not support any compression format + +### common options + +Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details. + +### max_rows_in_memory [int] + +When File Format is Excel,The maximum number of data items that can be cached in the memory. + +### sheet_name [string] + +Writer the sheet of the workbook + +## Example + +For text file format with `have_partition` and `custom_filename` and `sink_columns` + +```hocon + + CosFile { + path="/sink" + bucket = "cosn://seatunnel-test-1259587829" + secret_id = "xxxxxxxxxxxxxxxxxxx" + secret_key = "xxxxxxxxxxxxxxxxxxx" + region = "ap-chengdu" + file_format_type = "text" + field_delimiter = "\t" + row_delimiter = "\n" + have_partition = true + partition_by = ["age"] + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + custom_filename = true + file_name_expression = "${transactionId}_${now}" + filename_time_format = "yyyy.MM.dd" + sink_columns = ["name","age"] + is_enable_transaction = true + } + +``` + +For parquet file format with `have_partition` and `sink_columns` + +```hocon + + CosFile { + path="/sink" + bucket = "cosn://seatunnel-test-1259587829" + secret_id = "xxxxxxxxxxxxxxxxxxx" + secret_key = "xxxxxxxxxxxxxxxxxxx" + region = "ap-chengdu" + have_partition = true + partition_by = ["age"] + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_format_type = "parquet" + sink_columns = ["name","age"] + } + +``` + +For orc file format simple config + +```bash + + CosFile { + path="/sink" + bucket = "cosn://seatunnel-test-1259587829" + secret_id = "xxxxxxxxxxxxxxxxxxx" + secret_key = "xxxxxxxxxxxxxxxxxxx" + region = "ap-chengdu" + file_format_type = "orc" + } + +``` + +## Changelog + +### next version + +- Add file cos sink connector ([4979](https://github.com/apache/seatunnel/pull/4979)) + diff --git a/docs/en/connector-v2/source/CosFile.md b/docs/en/connector-v2/source/CosFile.md new file mode 100644 index 000000000000..18fc0299c9e2 --- /dev/null +++ b/docs/en/connector-v2/source/CosFile.md @@ -0,0 +1,289 @@ +# CosFile + +> Cos file source connector + +## Description + +Read data from aliyun Cos file system. + +:::tip + +If you use spark/flink, In order to use this connector, You must ensure your spark/flink cluster already integrated hadoop. The tested hadoop version is 2.x. + +If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you download and install SeaTunnel Engine. You can check the jar package under ${SEATUNNEL_HOME}/lib to confirm this. + +To use this connector you need put hadoop-cos-{hadoop.version}-{version}.jar and cos_api-bundle-{version}.jar in ${SEATUNNEL_HOME}/lib dir, download: [Hadoop-Cos-release](https://github.com/tencentyun/hadoop-cos/releases). It only supports hadoop version 2.6.5+ and version 8.0.2+. + +::: + +## Key features + +- [x] [batch](../../concept/connector-v2-features.md) +- [ ] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) + +Read all the data in a split in a pollNext call. What splits are read will be saved in snapshot. + +- [x] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [ ] [support user-defined split](../../concept/connector-v2-features.md) +- [x] file format type + - [x] text + - [x] csv + - [x] parquet + - [x] orc + - [x] json + - [x] excel + +## Options + +| name | type | required | default value | +|---------------------------|---------|----------|---------------------| +| path | string | yes | - | +| file_format_type | string | yes | - | +| bucket | string | yes | - | +| secret_id | string | yes | - | +| secret_key | string | yes | - | +| region | string | yes | - | +| read_columns | list | yes | - | +| delimiter | string | no | \001 | +| parse_partition_from_path | boolean | no | true | +| skip_header_row_number | long | no | 0 | +| date_format | string | no | yyyy-MM-dd | +| datetime_format | string | no | yyyy-MM-dd HH:mm:ss | +| time_format | string | no | HH:mm:ss | +| schema | config | no | - | +| common-options | | no | - | +| sheet_name | string | no | - | + +### path [string] + +The source file path. + +### delimiter [string] + +Field delimiter, used to tell connector how to slice and dice fields when reading text files + +default `\001`, the same as hive's default delimiter + +### parse_partition_from_path [boolean] + +Control whether parse the partition keys and values from file path + +For example if you read a file from path `cosn://hadoop-cluster/tmp/seatunnel/parquet/name=tyrantlucifer/age=26` + +Every record data from file will be added these two fields: + +| name | age | +|---------------|-----| +| tyrantlucifer | 26 | + +Tips: **Do not define partition fields in schema option** + +### date_format [string] + +Date type format, used to tell connector how to convert string to date, supported as the following formats: + +`yyyy-MM-dd` `yyyy.MM.dd` `yyyy/MM/dd` + +default `yyyy-MM-dd` + +### datetime_format [string] + +Datetime type format, used to tell connector how to convert string to datetime, supported as the following formats: + +`yyyy-MM-dd HH:mm:ss` `yyyy.MM.dd HH:mm:ss` `yyyy/MM/dd HH:mm:ss` `yyyyMMddHHmmss` + +default `yyyy-MM-dd HH:mm:ss` + +### time_format [string] + +Time type format, used to tell connector how to convert string to time, supported as the following formats: + +`HH:mm:ss` `HH:mm:ss.SSS` + +default `HH:mm:ss` + +### skip_header_row_number [long] + +Skip the first few lines, but only for the txt and csv. + +For example, set like following: + +`skip_header_row_number = 2` + +then SeaTunnel will skip the first 2 lines from source files + +### file_format_type [string] + +File type, supported as the following file types: + +`text` `csv` `parquet` `orc` `json` `excel` + +If you assign file type to `json`, you should also assign schema option to tell connector how to parse data to the row you want. + +For example: + +upstream data is the following: + +```json + +{"code": 200, "data": "get success", "success": true} + +``` + +You can also save multiple pieces of data in one file and split them by newline: + +```json lines + +{"code": 200, "data": "get success", "success": true} +{"code": 300, "data": "get failed", "success": false} + +``` + +you should assign schema as the following: + +```hocon + +schema { + fields { + code = int + data = string + success = boolean + } +} + +``` + +connector will generate data as the following: + +| code | data | success | +|------|-------------|---------| +| 200 | get success | true | + +If you assign file type to `parquet` `orc`, schema option not required, connector can find the schema of upstream data automatically. + +If you assign file type to `text` `csv`, you can choose to specify the schema information or not. + +For example, upstream data is the following: + +```text + +tyrantlucifer#26#male + +``` + +If you do not assign data schema connector will treat the upstream data as the following: + +| content | +|-----------------------| +| tyrantlucifer#26#male | + +If you assign data schema, you should also assign the option `delimiter` too except CSV file type + +you should assign schema and delimiter as the following: + +```hocon + +delimiter = "#" +schema { + fields { + name = string + age = int + gender = string + } +} + +``` + +connector will generate data as the following: + +| name | age | gender | +|---------------|-----|--------| +| tyrantlucifer | 26 | male | + +### bucket [string] + +The bucket address of Cos file system, for example: `Cos://tyrantlucifer-image-bed` + +### secret_id [string] + +The secret id of Cos file system. + +### secret_key [string] + +The secret key of Cos file system. + +### region [string] + +The region of cos file system. + +### schema [config] + +#### fields [Config] + +The schema of upstream data. + +### read_columns [list] + +The read column list of the data source, user can use it to implement field projection. + +The file type supported column projection as the following shown: + +- text +- json +- csv +- orc +- parquet +- excel + +**Tips: If the user wants to use this feature when reading `text` `json` `csv` files, the schema option must be configured** + +### common options + +Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. + +### sheet_name [string] + +Reader the sheet of the workbook,Only used when file_format is excel. + +## Example + +```hocon + + CosFile { + path = "/seatunnel/orc" + bucket = "cosn://seatunnel-test-1259587829" + secret_id = "xxxxxxxxxxxxxxxxxxx" + secret_key = "xxxxxxxxxxxxxxxxxxx" + region = "ap-chengdu" + file_format_type = "orc" + } + +``` + +```hocon + + CosFile { + path = "/seatunnel/json" + bucket = "cosn://seatunnel-test-1259587829" + secret_id = "xxxxxxxxxxxxxxxxxxx" + secret_key = "xxxxxxxxxxxxxxxxxxx" + region = "ap-chengdu" + file_format_type = "json" + schema { + fields { + id = int + name = string + } + } + } + +``` + +## Changelog + +### next version + +- Add file cos source connector ([4979](https://github.com/apache/seatunnel/pull/4979)) + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 3943159aefaf..a1c4e40fbb08 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -49,6 +49,8 @@ seatunnel.source.OssFile = connector-file-oss seatunnel.sink.OssFile = connector-file-oss seatunnel.source.OssJindoFile = connector-file-oss-jindo seatunnel.sink.OssJindoFile = connector-file-oss-jindo +seatunnel.source.CosFile = connector-file-cos +seatunnel.sink.CosFile = connector-file-cos seatunnel.source.Pulsar = connector-pulsar seatunnel.source.Hudi = connector-hudi seatunnel.sink.DingTalk = connector-dingtalk diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileSystemType.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileSystemType.java index 8d50cee46978..3d3965b7c3ff 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileSystemType.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/FileSystemType.java @@ -24,6 +24,7 @@ public enum FileSystemType implements Serializable { LOCAL("LocalFile"), OSS("OssFile"), OSS_JINDO("OssJindoFile"), + COS("CosFile"), FTP("FtpFile"), SFTP("SftpFile"), S3("S3File"); diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/pom.xml b/seatunnel-connectors-v2/connector-file/connector-file-cos/pom.xml new file mode 100644 index 000000000000..457357ad81f2 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/pom.xml @@ -0,0 +1,64 @@ + + + + 4.0.0 + + org.apache.seatunnel + connector-file + ${revision} + + + connector-file-cos + SeaTunnel : Connectors V2 : File : Cos + + + 2.6.5-8.0.2 + + + + + + org.apache.seatunnel + connector-file-base + ${project.version} + + + + org.apache.flink + flink-shaded-hadoop-2 + provided + + + org.apache.avro + avro + + + + + + com.qcloud.cos + hadoop-cos + ${hadoop-cos.version} + provided + + + + diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/config/CosConf.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/config/CosConf.java new file mode 100644 index 000000000000..211c24536872 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/config/CosConf.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.file.cos.config; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; + +import org.apache.hadoop.fs.CosNConfigKeys; + +import java.util.HashMap; + +public class CosConf extends HadoopConf { + private static final String HDFS_IMPL = "org.apache.hadoop.fs.CosFileSystem"; + private static final String SCHEMA = "cosn"; + + @Override + public String getFsHdfsImpl() { + return HDFS_IMPL; + } + + @Override + public String getSchema() { + return SCHEMA; + } + + public CosConf(String hdfsNameKey) { + super(hdfsNameKey); + } + + public static HadoopConf buildWithConfig(Config config) { + HadoopConf hadoopConf = new CosConf(config.getString(CosConfig.BUCKET.key())); + HashMap cosOptions = new HashMap<>(); + cosOptions.put( + CosNConfigKeys.COSN_USERINFO_SECRET_ID_KEY, + config.getString(CosConfig.SECRET_ID.key())); + cosOptions.put( + CosNConfigKeys.COSN_USERINFO_SECRET_KEY_KEY, + config.getString(CosConfig.SECRET_KEY.key())); + cosOptions.put(CosNConfigKeys.COSN_REGION_KEY, config.getString(CosConfig.REGION.key())); + hadoopConf.setExtraOptions(cosOptions); + return hadoopConf; + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/config/CosConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/config/CosConfig.java new file mode 100644 index 000000000000..cbbd68ef7dc7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/config/CosConfig.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.file.cos.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; + +public class CosConfig extends BaseSourceConfig { + public static final Option SECRET_ID = + Options.key("secret_id") + .stringType() + .noDefaultValue() + .withDescription("COS bucket secret id"); + public static final Option SECRET_KEY = + Options.key("secret_key") + .stringType() + .noDefaultValue() + .withDescription("COS bucket secret key"); + public static final Option REGION = + Options.key("region").stringType().noDefaultValue().withDescription("COS region"); + public static final Option BUCKET = + Options.key("bucket").stringType().noDefaultValue().withDescription("COS bucket"); +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/sink/CosFileSink.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/sink/CosFileSink.java new file mode 100644 index 000000000000..bfc6fa4ff10f --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/sink/CosFileSink.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.file.cos.sink; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.common.PrepareFailException; +import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; +import org.apache.seatunnel.api.sink.SeaTunnelSink; +import org.apache.seatunnel.common.config.CheckConfigUtil; +import org.apache.seatunnel.common.config.CheckResult; +import org.apache.seatunnel.common.constants.PluginType; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.cos.config.CosConf; +import org.apache.seatunnel.connectors.seatunnel.file.cos.config.CosConfig; +import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; +import org.apache.seatunnel.connectors.seatunnel.file.sink.BaseFileSink; + +import com.google.auto.service.AutoService; + +@AutoService(SeaTunnelSink.class) +public class CosFileSink extends BaseFileSink { + @Override + public String getPluginName() { + return FileSystemType.COS.getFileSystemPluginName(); + } + + @Override + public void prepare(Config pluginConfig) throws PrepareFailException { + super.prepare(pluginConfig); + CheckResult result = + CheckConfigUtil.checkAllExists( + pluginConfig, + CosConfig.FILE_PATH.key(), + CosConfig.REGION.key(), + CosConfig.SECRET_ID.key(), + CosConfig.SECRET_KEY.key(), + CosConfig.BUCKET.key()); + if (!result.isSuccess()) { + throw new FileConnectorException( + SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, + String.format( + "PluginName: %s, PluginType: %s, Message: %s", + getPluginName(), PluginType.SINK, result.getMsg())); + } + hadoopConf = CosConf.buildWithConfig(pluginConfig); + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/sink/CosFileSinkFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/sink/CosFileSinkFactory.java new file mode 100644 index 000000000000..9de5386bc6b2 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/sink/CosFileSinkFactory.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.file.cos.sink; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSinkConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.cos.config.CosConfig; + +import com.google.auto.service.AutoService; + +@AutoService(Factory.class) +public class CosFileSinkFactory implements TableSinkFactory { + @Override + public String factoryIdentifier() { + return FileSystemType.COS.getFileSystemPluginName(); + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required(CosConfig.FILE_PATH) + .required(CosConfig.BUCKET) + .required(CosConfig.SECRET_ID) + .required(CosConfig.SECRET_KEY) + .required(CosConfig.REGION) + .optional(BaseSinkConfig.FILE_FORMAT_TYPE) + .conditional( + BaseSinkConfig.FILE_FORMAT_TYPE, + FileFormat.TEXT, + BaseSinkConfig.ROW_DELIMITER, + BaseSinkConfig.FIELD_DELIMITER, + BaseSinkConfig.TXT_COMPRESS) + .conditional( + BaseSinkConfig.FILE_FORMAT_TYPE, + FileFormat.CSV, + BaseSinkConfig.TXT_COMPRESS) + .conditional( + BaseSinkConfig.FILE_FORMAT_TYPE, + FileFormat.JSON, + BaseSinkConfig.TXT_COMPRESS) + .conditional( + BaseSinkConfig.FILE_FORMAT_TYPE, + FileFormat.ORC, + BaseSinkConfig.ORC_COMPRESS) + .conditional( + BaseSinkConfig.FILE_FORMAT_TYPE, + FileFormat.PARQUET, + BaseSinkConfig.PARQUET_COMPRESS) + .optional(BaseSinkConfig.CUSTOM_FILENAME) + .conditional( + BaseSinkConfig.CUSTOM_FILENAME, + true, + BaseSinkConfig.FILE_NAME_EXPRESSION, + BaseSinkConfig.FILENAME_TIME_FORMAT) + .optional(BaseSinkConfig.HAVE_PARTITION) + .conditional( + BaseSinkConfig.HAVE_PARTITION, + true, + BaseSinkConfig.PARTITION_BY, + BaseSinkConfig.PARTITION_DIR_EXPRESSION, + BaseSinkConfig.IS_PARTITION_FIELD_WRITE_IN_FILE) + .optional(BaseSinkConfig.SINK_COLUMNS) + .optional(BaseSinkConfig.IS_ENABLE_TRANSACTION) + .optional(BaseSinkConfig.DATE_FORMAT) + .optional(BaseSinkConfig.DATETIME_FORMAT) + .optional(BaseSinkConfig.TIME_FORMAT) + .build(); + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSource.java new file mode 100644 index 000000000000..aefc339121e0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSource.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.file.cos.source; + +import org.apache.seatunnel.shade.com.typesafe.config.Config; + +import org.apache.seatunnel.api.common.PrepareFailException; +import org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.config.CheckConfigUtil; +import org.apache.seatunnel.common.config.CheckResult; +import org.apache.seatunnel.common.constants.PluginType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.cos.config.CosConf; +import org.apache.seatunnel.connectors.seatunnel.file.cos.config.CosConfig; +import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; +import org.apache.seatunnel.connectors.seatunnel.file.source.BaseFileSource; +import org.apache.seatunnel.connectors.seatunnel.file.source.reader.ReadStrategyFactory; + +import com.google.auto.service.AutoService; + +import java.io.IOException; + +@AutoService(SeaTunnelSource.class) +public class CosFileSource extends BaseFileSource { + @Override + public String getPluginName() { + return FileSystemType.COS.getFileSystemPluginName(); + } + + @Override + public void prepare(Config pluginConfig) throws PrepareFailException { + CheckResult result = + CheckConfigUtil.checkAllExists( + pluginConfig, + CosConfig.FILE_PATH.key(), + CosConfig.FILE_FORMAT_TYPE.key(), + CosConfig.SECRET_ID.key(), + CosConfig.SECRET_KEY.key(), + CosConfig.REGION.key(), + CosConfig.BUCKET.key()); + if (!result.isSuccess()) { + throw new FileConnectorException( + SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, + String.format( + "PluginName: %s, PluginType: %s, Message: %s", + getPluginName(), PluginType.SOURCE, result.getMsg())); + } + readStrategy = + ReadStrategyFactory.of(pluginConfig.getString(CosConfig.FILE_FORMAT_TYPE.key())); + readStrategy.setPluginConfig(pluginConfig); + String path = pluginConfig.getString(CosConfig.FILE_PATH.key()); + hadoopConf = CosConf.buildWithConfig(pluginConfig); + try { + filePaths = readStrategy.getFileNamesByPath(hadoopConf, path); + } catch (IOException e) { + String errorMsg = String.format("Get file list from this path [%s] failed", path); + throw new FileConnectorException( + FileConnectorErrorCode.FILE_LIST_GET_FAILED, errorMsg, e); + } + // support user-defined schema + FileFormat fileFormat = + FileFormat.valueOf( + pluginConfig.getString(CosConfig.FILE_FORMAT_TYPE.key()).toUpperCase()); + // only json text csv type support user-defined schema now + if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + switch (fileFormat) { + case CSV: + case TEXT: + case JSON: + case EXCEL: + SeaTunnelRowType userDefinedSchema = + CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); + readStrategy.setSeaTunnelRowTypeInfo(userDefinedSchema); + rowType = readStrategy.getActualSeaTunnelRowTypeInfo(); + break; + case ORC: + case PARQUET: + throw new FileConnectorException( + CommonErrorCode.UNSUPPORTED_OPERATION, + "SeaTunnel does not support user-defined schema for [parquet, orc] files"); + default: + // never got in there + throw new FileConnectorException( + CommonErrorCode.ILLEGAL_ARGUMENT, + "SeaTunnel does not supported this file format"); + } + } else { + try { + rowType = readStrategy.getSeaTunnelRowTypeInfo(hadoopConf, filePaths.get(0)); + } catch (FileConnectorException e) { + String errorMsg = + String.format("Get table schema from file [%s] failed", filePaths.get(0)); + throw new FileConnectorException( + CommonErrorCode.TABLE_SCHEMA_GET_FAILED, errorMsg, e); + } + } + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java new file mode 100644 index 000000000000..d0b781f1a144 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.file.cos.source; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileFormat; +import org.apache.seatunnel.connectors.seatunnel.file.config.FileSystemType; +import org.apache.seatunnel.connectors.seatunnel.file.cos.config.CosConfig; + +import com.google.auto.service.AutoService; + +import java.util.Arrays; + +@AutoService(Factory.class) +public class CosFileSourceFactory implements TableSourceFactory { + @Override + public String factoryIdentifier() { + return FileSystemType.OSS.getFileSystemPluginName(); + } + + @Override + public OptionRule optionRule() { + return OptionRule.builder() + .required(CosConfig.FILE_PATH) + .required(CosConfig.BUCKET) + .required(CosConfig.SECRET_ID) + .required(CosConfig.SECRET_KEY) + .required(CosConfig.REGION) + .required(BaseSourceConfig.FILE_FORMAT_TYPE) + .conditional( + BaseSourceConfig.FILE_FORMAT_TYPE, + FileFormat.TEXT, + BaseSourceConfig.DELIMITER) + .conditional( + BaseSourceConfig.FILE_FORMAT_TYPE, + Arrays.asList( + FileFormat.TEXT, FileFormat.JSON, FileFormat.EXCEL, FileFormat.CSV), + CatalogTableUtil.SCHEMA) + .optional(BaseSourceConfig.PARSE_PARTITION_FROM_PATH) + .optional(BaseSourceConfig.DATE_FORMAT) + .optional(BaseSourceConfig.DATETIME_FORMAT) + .optional(BaseSourceConfig.TIME_FORMAT) + .build(); + } + + @Override + public Class getSourceClass() { + return CosFileSource.class; + } +} diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/resources/services/org.apache.hadoop.fs.FileSystem b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/resources/services/org.apache.hadoop.fs.FileSystem new file mode 100644 index 000000000000..b4ecb7e0c7e3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/resources/services/org.apache.hadoop.fs.FileSystem @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.hadoop.fs.CosFileSystem \ No newline at end of file diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/cos/CosFileFactoryTest.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/cos/CosFileFactoryTest.java new file mode 100644 index 000000000000..6691f5b1f2af --- /dev/null +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/test/java/org/apache/seatunnel/connectors/seatunnel/file/cos/CosFileFactoryTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.file.cos; + +import org.apache.seatunnel.connectors.seatunnel.file.cos.sink.CosFileSinkFactory; +import org.apache.seatunnel.connectors.seatunnel.file.cos.source.CosFileSourceFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class CosFileFactoryTest { + + @Test + void optionRule() { + Assertions.assertNotNull((new CosFileSourceFactory()).optionRule()); + Assertions.assertNotNull((new CosFileSinkFactory()).optionRule()); + } +} diff --git a/seatunnel-connectors-v2/connector-file/pom.xml b/seatunnel-connectors-v2/connector-file/pom.xml index d20e6296cbfc..4bdfa981cee6 100644 --- a/seatunnel-connectors-v2/connector-file/pom.xml +++ b/seatunnel-connectors-v2/connector-file/pom.xml @@ -39,6 +39,7 @@ connector-file-sftp connector-file-s3 connector-file-jindo-oss + connector-file-cos diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index fa1ac63c0bf5..acc6a4fc3215 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -278,6 +278,12 @@ ${project.version} provided + + org.apache.seatunnel + connector-file-cos + ${project.version} + provided + org.apache.seatunnel connector-file-ftp diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/pom.xml new file mode 100644 index 000000000000..aa51e1cc820b --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/pom.xml @@ -0,0 +1,48 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-file-cos-e2e + SeaTunnel : E2E : Connector V2 : File Cos + + + + org.apache.seatunnel + connector-fake + ${project.version} + test + + + org.apache.seatunnel + connector-file-cos + ${project.version} + test + + + org.apache.seatunnel + connector-assert + ${project.version} + test + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/cos/CosFileIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/cos/CosFileIT.java new file mode 100644 index 000000000000..aaa2c1a2763e --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/cos/CosFileIT.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.connector.file.cos; + +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.TestContainer; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; + +import java.io.IOException; + +@Disabled +public class CosFileIT extends TestSuiteBase { + + @TestTemplate + public void testCosFileWriteAndRead(TestContainer container) + throws IOException, InterruptedException { + // test cos excel file + Container.ExecResult excelWriteResult = + container.executeJob("/excel/fake_to_cos_excel.conf"); + Assertions.assertEquals(0, excelWriteResult.getExitCode(), excelWriteResult.getStderr()); + Container.ExecResult excelReadResult = + container.executeJob("/excel/cos_excel_to_assert.conf"); + Assertions.assertEquals(0, excelReadResult.getExitCode(), excelReadResult.getStderr()); + + // test cos text file + Container.ExecResult textWriteResult = + container.executeJob("/text/fake_to_cos_file_text.conf"); + Assertions.assertEquals(0, textWriteResult.getExitCode()); + Container.ExecResult textReadResult = + container.executeJob("/text/cos_file_text_to_assert.conf"); + Assertions.assertEquals(0, textReadResult.getExitCode()); + + // test cos json file + Container.ExecResult jsonWriteResult = + container.executeJob("/json/fake_to_cos_file_json.conf"); + Assertions.assertEquals(0, jsonWriteResult.getExitCode()); + Container.ExecResult jsonReadResult = + container.executeJob("/json/cos_file_json_to_assert.conf"); + Assertions.assertEquals(0, jsonReadResult.getExitCode()); + + // test cos orc file + Container.ExecResult orcWriteResult = + container.executeJob("/orc/fake_to_cos_file_orc.conf"); + Assertions.assertEquals(0, orcWriteResult.getExitCode()); + Container.ExecResult orcReadResult = + container.executeJob("/orc/cos_file_orc_to_assert.conf"); + Assertions.assertEquals(0, orcReadResult.getExitCode()); + + // test cos parquet file + Container.ExecResult parquetWriteResult = + container.executeJob("/parquet/fake_to_cos_file_parquet.conf"); + Assertions.assertEquals(0, parquetWriteResult.getExitCode()); + Container.ExecResult parquetReadResult = + container.executeJob("/parquet/cos_file_parquet_to_assert.conf"); + Assertions.assertEquals(0, parquetReadResult.getExitCode()); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/excel/cos_excel_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/excel/cos_excel_to_assert.conf new file mode 100644 index 000000000000..b71709318ec5 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/excel/cos_excel_to_assert.conf @@ -0,0 +1,116 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + CosFile { + path = "/read/excel" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + result_table_name = "fake" + file_format_type = excel + delimiter = ; + skip_header_row_number = 1 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/excel/fake_to_cos_excel.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/excel/fake_to_cos_excel.conf new file mode 100644 index 000000000000..4c603f5633cc --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/excel/fake_to_cos_excel.conf @@ -0,0 +1,82 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + result_table_name = "fake" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + CosFile { + path="/sink/execl" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "excel" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/json/cos_file_json_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/json/cos_file_json_to_assert.conf new file mode 100644 index 000000000000..d88761799b13 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/json/cos_file_json_to_assert.conf @@ -0,0 +1,114 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + CosFile { + path = "/read/json" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + file_format_type = "json" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + C_MAP = "map" + C_ARRAY = "array" + C_STRING = string + C_BOOLEAN = boolean + C_TINYINT = tinyint + C_SMALLINT = smallint + C_INT = int + C_BIGINT = bigint + C_FLOAT = float + C_DOUBLE = double + C_BYTES = bytes + C_DATE = date + C_DECIMAL = "decimal(38, 18)" + C_TIMESTAMP = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/json/fake_to_cos_file_json.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/json/fake_to_cos_file_json.conf new file mode 100644 index 000000000000..20f54863d6d7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/json/fake_to_cos_file_json.conf @@ -0,0 +1,83 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + CosFile { + path="/sink/json" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "json" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/orc/cos_file_orc_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/orc/cos_file_orc_to_assert.conf new file mode 100644 index 000000000000..1041997ed684 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/orc/cos_file_orc_to_assert.conf @@ -0,0 +1,80 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + CosFile { + path = "/read/orc" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + file_format_type = "orc" + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/orc/fake_to_cos_file_orc.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/orc/fake_to_cos_file_orc.conf new file mode 100644 index 000000000000..879993b4ea9e --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/orc/fake_to_cos_file_orc.conf @@ -0,0 +1,84 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + CosFile { + path="/sink/orc" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "orc" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "zlib" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/parquet/cos_file_parquet_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/parquet/cos_file_parquet_to_assert.conf new file mode 100644 index 000000000000..8bf9c171ce82 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/parquet/cos_file_parquet_to_assert.conf @@ -0,0 +1,80 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + CosFile { + path = "/read/parquet" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + file_format_type = "parquet" + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/parquet/fake_to_cos_file_parquet.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/parquet/fake_to_cos_file_parquet.conf new file mode 100644 index 000000000000..bb86e5f8b2e1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/parquet/fake_to_cos_file_parquet.conf @@ -0,0 +1,84 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + CosFile { + path="/sink/parquet" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "parquet" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "gzip" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/text/cos_file_text_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/text/cos_file_text_to_assert.conf new file mode 100644 index 000000000000..d53a046079ce --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/text/cos_file_text_to_assert.conf @@ -0,0 +1,114 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + CosFile { + path = "/read/text" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + file_format_type = "text" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/text/fake_to_cos_file_text.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/text/fake_to_cos_file_text.conf new file mode 100644 index 000000000000..f93af2e212ea --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-cos-e2e/src/test/resources/text/fake_to_cos_file_text.conf @@ -0,0 +1,84 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + FakeSource { + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + result_table_name = "fake" + } +} + +sink { + CosFile { + path="/sink/text" + bucket = "cosn://seatunnel-test" + secret_id = "dummy" + secret_key = "dummy" + region = "ap-chengdu" + row_delimiter = "\n" + partition_dir_expression = "${k0}=${v0}" + is_partition_field_write_in_file = true + file_name_expression = "${transactionId}_${now}" + file_format_type = "text" + filename_time_format = "yyyy.MM.dd" + is_enable_transaction = true + compress_codec = "lzo" + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 65798fee100c..8644b551b2f5 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -36,6 +36,7 @@ connector-influxdb-e2e connector-amazondynamodb-e2e connector-file-local-e2e + connector-file-cos-e2e connector-file-sftp-e2e connector-cassandra-e2e connector-neo4j-e2e From 3e9b9f5849a1829ec404fee081fc6ee43f2aa9fc Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Thu, 27 Jul 2023 09:37:54 +0800 Subject: [PATCH 22/43] [Fix][Zeta] Fix SinkFlowLifeCycle without init lastCommitInfo (#5152) --- .../seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java | 1 + 1 file changed, 1 insertion(+) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java index fe1e5c5a8944..9e67a601eb9a 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/task/flow/SinkFlowLifeCycle.java @@ -109,6 +109,7 @@ public SinkFlowLifeCycle( public void init() throws Exception { this.writerStateSerializer = sinkAction.getSink().getWriterStateSerializer(); this.committer = sinkAction.getSink().createCommitter(); + this.lastCommitInfo = Optional.empty(); } @Override From 4b4b5f96404ca4ca2a8cab385c67003348bf8bd9 Mon Sep 17 00:00:00 2001 From: monster <60029759+MonsterChenzhuo@users.noreply.github.com> Date: Thu, 27 Jul 2023 10:06:10 +0800 Subject: [PATCH 23/43] [Hotfix][MongodbCDC]Refine data format to adapt to universal logic (#5162) Co-authored-by: chenzy15 --- docs/en/connector-v2/source/MongoDB-CDC.md | 41 ++++++++++++++++--- .../IncrementalSourceStreamFetcher.java | 14 +++---- .../mongodb/config/MongodbSourceOptions.java | 2 + .../cdc/mongodb/utils/MongodbRecordUtils.java | 9 ++++ .../cdc/mongodb/utils/ResumeToken.java | 11 +++-- 5 files changed, 61 insertions(+), 16 deletions(-) diff --git a/docs/en/connector-v2/source/MongoDB-CDC.md b/docs/en/connector-v2/source/MongoDB-CDC.md index cb7c2f32acf5..d78f70110fc1 100644 --- a/docs/en/connector-v2/source/MongoDB-CDC.md +++ b/docs/en/connector-v2/source/MongoDB-CDC.md @@ -84,8 +84,8 @@ The following table lists the field data type mapping from MongoDB BSON type to | Int64 | BIGINT | | Double | DOUBLE | | Decimal128 | DECIMAL | -| Date | Date | -| Timestamp | Timestamp | +| Date | DATE | +| Timestamp | TIMESTAMP | | Object | ROW | | Array | ARRAY | @@ -274,9 +274,38 @@ sink { } ``` -## Changelog +## Format of real-time streaming data -- [Feature]Add MongoDB CDC Source Connector([4923](https://github.com/apache/seatunnel/pull/4923)) - -### next version +```shell +{ + _id : { }, // Identifier of the open change stream, can be assigned to the 'resumeAfter' parameter for subsequent resumption of this change stream + "operationType" : "", // The type of change operation that occurred, such as: insert, delete, update, etc. + "fullDocument" : { }, // The full document data involved in the change operation. This field does not exist in delete operations + "ns" : { + "db" : "", // The database where the change operation occurred + "coll" : "" // The collection where the change operation occurred + }, + "to" : { // These fields are displayed only when the operation type is 'rename' + "db" : "", // The new database name after the change + "coll" : "" // The new collection name after the change + }, + "source":{ + "ts_ms":"", // The timestamp when the change operation occurred + "table":"" // The collection where the change operation occurred + "db":"", // The database where the change operation occurred + "snapshot":"false" // Identify the current stage of data synchronization + }, + "documentKey" : { "_id" : }, // The _id field value of the document involved in the change operation + "updateDescription" : { // Description of the update operation + "updatedFields" : { }, // The fields and values that the update operation modified + "removedFields" : [ "", ... ] // The fields and values that the update operation removed + } + "clusterTime" : , // The timestamp of the Oplog log entry corresponding to the change operation + "txnNumber" : , // If the change operation is executed in a multi-document transaction, this field and value are displayed, representing the transaction number + "lsid" : { // Represents information related to the Session in which the transaction is located + "id" : , + "uid" : + } +} +``` diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java index c4d5fdfd68fe..5257064dc1fe 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java @@ -28,6 +28,7 @@ import com.google.common.util.concurrent.ThreadFactoryBuilder; import io.debezium.connector.base.ChangeEventQueue; import io.debezium.pipeline.DataChangeEvent; +import io.debezium.relational.TableId; import lombok.extern.slf4j.Slf4j; import java.util.ArrayList; @@ -38,6 +39,8 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import static org.apache.seatunnel.connectors.cdc.base.utils.SourceRecordUtils.getTableId; + /** * Fetcher to fetch data from table split, the split is the incremental split {@link * IncrementalSplit}. @@ -147,14 +150,11 @@ public void close() { private boolean shouldEmit(SourceRecord sourceRecord) { if (taskContext.isDataChangeRecord(sourceRecord)) { Offset position = taskContext.getStreamOffset(sourceRecord); - // TODO: The sourceRecord from MongoDB CDC and MySQL CDC are inconsistent. For - // compatibility, the getTableId method is commented out for now. - // TableId tableId = getTableId(sourceRecord); + TableId tableId = getTableId(sourceRecord); if (!taskContext.isExactlyOnce()) { - // log.trace( - // "The table {} is not support exactly-once, so ignore the - // watermark check", - // tableId); + log.trace( + "The table {} is not support exactly-once, so ignore the watermark check", + tableId); return position.isAfter(splitStartWatermark); } // TODO only the table who captured snapshot splits need to filter( Used to support diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java index dac939777f7f..170bef34e961 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java @@ -108,6 +108,8 @@ public class MongodbSourceOptions extends SourceOptions { + " { \"name\": \"source\"," + " \"type\": [{\"name\": \"source\", \"type\": \"record\", \"fields\": [" + " {\"name\": \"ts_ms\", \"type\": \"long\"}," + + " {\"name\": \"table\", \"type\": [\"string\", \"null\"]}," + + " {\"name\": \"db\", \"type\": [\"string\", \"null\"]}," + " {\"name\": \"snapshot\", \"type\": [\"string\", \"null\"] } ]" + " }, \"null\" ] }," + " { \"name\": \"ts_ms\", \"type\": [\"long\", \"null\"]}," diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java index 84af2f7fda62..c4d51c59e419 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java @@ -24,6 +24,7 @@ import org.bson.BsonDocument; import org.bson.BsonTimestamp; +import org.bson.BsonValue; import org.bson.json.JsonWriterSettings; import com.mongodb.kafka.connect.source.json.formatter.DefaultJson; @@ -39,6 +40,7 @@ import java.util.Map; import static com.mongodb.kafka.connect.source.schema.AvroSchema.fromJson; +import static io.debezium.connector.AbstractSourceInfo.TABLE_NAME_KEY; import static org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkEvent.isWatermarkEvent; import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.COLL_FIELD; import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DB_FIELD; @@ -46,6 +48,7 @@ import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.NS_FIELD; import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.OUTPUT_SCHEMA; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SOURCE_FIELD; public class MongodbRecordUtils { @@ -117,6 +120,12 @@ public static String getOffsetValue(@Nonnull SourceRecord sourceRecord, String k SchemaAndValue keySchemaAndValue = schemaAndValue.toSchemaAndValue( fromJson(AvroSchemaDefaults.DEFAULT_AVRO_KEY_SCHEMA), keyDocument); + BsonDocument source = valueDocument.get(SOURCE_FIELD).asDocument(); + BsonValue table = valueDocument.get(NS_FIELD).asDocument().get(COLL_FIELD); + BsonValue db = valueDocument.get(NS_FIELD).asDocument().get(DB_FIELD); + source.append(TABLE_NAME_KEY, table); + source.append(DB_FIELD, db); + valueDocument.replace(SOURCE_FIELD, source); SchemaAndValue valueSchemaAndValue = schemaAndValue.toSchemaAndValue(fromJson(OUTPUT_SCHEMA), valueDocument); diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java index 5ee8962bc532..1ef6870c85e8 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java @@ -17,6 +17,8 @@ package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + import org.bson.BsonDocument; import org.bson.BsonTimestamp; import org.bson.BsonValue; @@ -27,6 +29,8 @@ import java.nio.ByteOrder; import java.util.Objects; +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; + public class ResumeToken { private static final int K_TIMESTAMP = 130; @@ -41,14 +45,15 @@ public static BsonTimestamp decodeTimestamp(BsonDocument resumeToken) { } else if (bsonValue.isString()) { // Hex-encoded string (v0 or v1) keyStringBytes = hexToUint8Array(bsonValue.asString().getValue()); } else { - throw new IllegalArgumentException( - "Unknown resume token format: " + resumeToken.toJson()); + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Unknown resume token format: " + bsonValue); } ByteBuffer buffer = ByteBuffer.wrap(keyStringBytes).order(ByteOrder.BIG_ENDIAN); int kType = buffer.get() & 0xff; if (kType != K_TIMESTAMP) { - throw new IllegalArgumentException("Unknown keyType of timestamp: " + kType); + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Unknown keyType of timestamp: " + kType); } int t = buffer.getInt(); From 9821dcf4761be094b6024d47740d1caa3ccf39c8 Mon Sep 17 00:00:00 2001 From: David Zollo Date: Thu, 27 Jul 2023 11:55:36 +0800 Subject: [PATCH 24/43] [Chore] Update bug-report.yml (#5160) --- .github/ISSUE_TEMPLATE/bug-report.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/bug-report.yml b/.github/ISSUE_TEMPLATE/bug-report.yml index 39f5b87900e5..5892a2677fca 100644 --- a/.github/ISSUE_TEMPLATE/bug-report.yml +++ b/.github/ISSUE_TEMPLATE/bug-report.yml @@ -90,10 +90,10 @@ body: - type: textarea attributes: - label: Flink or Spark Version - description: Provide Flink or Spark Version. + label: Zeta or Flink or Spark Version + description: Provide Zeta or Flink or Spark Version. placeholder: > - Please provide the version of Flink or Spark. + Please provide the version of Zeta or Flink or Spark. validations: required: false From 0e4190ab2efddf2d3a1b498e22614f7828eec0fe Mon Sep 17 00:00:00 2001 From: happyboy1024 <137260654+happyboy1024@users.noreply.github.com> Date: Thu, 27 Jul 2023 11:58:15 +0800 Subject: [PATCH 25/43] [Improve][CDC] support exactly-once of cdc and fix the BinlogOffset comparing bug (#5057) * [Improve][CDC] support exactly-once of cdc, fix the BinlogOffset comparing bug * [Improve][CDC] adjust code style * [Improve][CDC] fix ci error --------- Co-authored-by: happyboy1024 <296442618@qq.com> --- .../IncrementalSourceScanFetcher.java | 9 +-- .../IncrementalSourceStreamFetcher.java | 77 ++++++++++++++++++- .../cdc/mysql/source/offset/BinlogOffset.java | 8 +- 3 files changed, 84 insertions(+), 10 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceScanFetcher.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceScanFetcher.java index 7a09ac6bc4ef..97c0c523e639 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceScanFetcher.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceScanFetcher.java @@ -223,14 +223,11 @@ public void close() { private boolean isChangeRecordInChunkRange(SourceRecord record) { if (taskContext.isDataChangeRecord(record)) { + // fix the between condition return taskContext.isRecordBetween( record, - null == currentSnapshotSplit.getSplitStart() - ? null - : new Object[] {currentSnapshotSplit.getSplitStart()}, - null == currentSnapshotSplit.getSplitEnd() - ? null - : new Object[] {currentSnapshotSplit.getSplitEnd()}); + currentSnapshotSplit.getSplitStart(), + currentSnapshotSplit.getSplitEnd()); } return false; } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java index 5257064dc1fe..2b8e9f7725fd 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-base/src/main/java/org/apache/seatunnel/connectors/cdc/base/source/reader/external/IncrementalSourceStreamFetcher.java @@ -19,6 +19,7 @@ import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; +import org.apache.seatunnel.connectors.cdc.base.source.split.CompletedSnapshotSplitInfo; import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceRecords; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; @@ -32,8 +33,12 @@ import lombok.extern.slf4j.Slf4j; import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; @@ -49,6 +54,8 @@ public class IncrementalSourceStreamFetcher implements Fetcher { private final FetchTask.Context taskContext; private final ExecutorService executorService; + // has entered pure binlog mode + private final Set pureBinlogPhaseTables; private volatile ChangeEventQueue queue; private volatile Throwable readException; @@ -58,6 +65,11 @@ public class IncrementalSourceStreamFetcher implements Fetcher maxSplitHighWatermarkMap; + // finished spilt info + private Map> finishedSplitsInfo; + private static final long READER_CLOSE_TIMEOUT_SECONDS = 30L; public IncrementalSourceStreamFetcher(FetchTask.Context taskContext, int subTaskId) { @@ -65,6 +77,7 @@ public IncrementalSourceStreamFetcher(FetchTask.Context taskContext, int subTask ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat("debezium-reader-" + subTaskId).build(); this.executorService = Executors.newSingleThreadExecutor(threadFactory); + this.pureBinlogPhaseTables = new HashSet<>(); } @Override @@ -157,14 +170,72 @@ private boolean shouldEmit(SourceRecord sourceRecord) { tableId); return position.isAfter(splitStartWatermark); } - // TODO only the table who captured snapshot splits need to filter( Used to support - // Exactly-Once ) - return position.isAfter(splitStartWatermark); + // check whether the pure binlog mode has been entered + if (hasEnterPureBinlogPhase(tableId, position)) { + return true; + } + // not enter pure binlog mode and need to check whether the current record meets the + // emitting conditions. + if (finishedSplitsInfo.containsKey(tableId)) { + for (CompletedSnapshotSplitInfo splitInfo : finishedSplitsInfo.get(tableId)) { + if (taskContext.isRecordBetween( + sourceRecord, + splitInfo.getSplitStart(), + splitInfo.getSplitEnd()) + && position.isAfter(splitInfo.getWatermark().getHighWatermark())) { + return true; + } + } + } + return false; } return true; } + private boolean hasEnterPureBinlogPhase(TableId tableId, Offset position) { + // only the table who captured snapshot splits need to filter + if (pureBinlogPhaseTables.contains(tableId)) { + return true; + } + // the existed tables those have finished snapshot reading + if (maxSplitHighWatermarkMap.containsKey(tableId) + && position.isAtOrAfter(maxSplitHighWatermarkMap.get(tableId))) { + pureBinlogPhaseTables.add(tableId); + return true; + } + return false; + } + private void configureFilter() { splitStartWatermark = currentIncrementalSplit.getStartupOffset(); + Map> splitsInfoMap = new HashMap<>(); + Map tableIdBinlogPositionMap = new HashMap<>(); + List completedSnapshotSplitInfos = + currentIncrementalSplit.getCompletedSnapshotSplitInfos(); + + // latest-offset mode + if (completedSnapshotSplitInfos.isEmpty()) { + for (TableId tableId : currentIncrementalSplit.getTableIds()) { + tableIdBinlogPositionMap.put(tableId, currentIncrementalSplit.getStartupOffset()); + } + } + + // calculate the max high watermark of every table + for (CompletedSnapshotSplitInfo finishedSplitInfo : completedSnapshotSplitInfos) { + TableId tableId = finishedSplitInfo.getTableId(); + List list = + splitsInfoMap.getOrDefault(tableId, new ArrayList<>()); + list.add(finishedSplitInfo); + splitsInfoMap.put(tableId, list); + + Offset highWatermark = finishedSplitInfo.getWatermark().getHighWatermark(); + Offset maxHighWatermark = tableIdBinlogPositionMap.get(tableId); + if (maxHighWatermark == null || highWatermark.isAfter(maxHighWatermark)) { + tableIdBinlogPositionMap.put(tableId, highWatermark); + } + } + this.finishedSplitsInfo = splitsInfoMap; + this.maxSplitHighWatermarkMap = tableIdBinlogPositionMap; + this.pureBinlogPhaseTables.clear(); } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffset.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffset.java index 195b1a5a7c6b..0d91c02fee7a 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffset.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/offset/BinlogOffset.java @@ -176,7 +176,13 @@ public int compareTo(Offset offset) { // compared ... long timestamp = this.getTimestamp(); long targetTimestamp = that.getTimestamp(); - return Long.compare(timestamp, targetTimestamp); + // Timestamps are presupposes that they exist, + // because timestamps do not exist for low watermark and high watermark. + // If not judging here results in the really binlog offset comparison to watermark + // always being true. + if (timestamp != 0 && targetTimestamp != 0) { + return Long.compare(timestamp, targetTimestamp); + } } // First compare the MySQL binlog filenames From ce39948ca5d1bfae35e4e4dc503d56dc1345cf18 Mon Sep 17 00:00:00 2001 From: Carl-Zhou-CN <67902676+Carl-Zhou-CN@users.noreply.github.com> Date: Fri, 28 Jul 2023 15:11:03 +0800 Subject: [PATCH 26/43] [Docs][Connector-V2][Hudi] Reconstruct the Hudi connector document (#4905) * [Docs][Connector-V2][Hudi] Reconstruct the Hudi connector document --------- Co-authored-by: zhouyao --- docs/en/connector-v2/source/Hudi.md | 82 ++++++++++++++++------------- 1 file changed, 44 insertions(+), 38 deletions(-) diff --git a/docs/en/connector-v2/source/Hudi.md b/docs/en/connector-v2/source/Hudi.md index cb3b154d58b6..b70d34608ea0 100644 --- a/docs/en/connector-v2/source/Hudi.md +++ b/docs/en/connector-v2/source/Hudi.md @@ -2,69 +2,67 @@ > Hudi source connector -## Description +## Support Those Engines -Used to read data from Hudi. Currently, only supports hudi cow table and Snapshot Query with Batch Mode. +> Spark
+> Flink
+> SeaTunnel Zeta
-In order to use this connector, You must ensure your spark/flink cluster already integrated hive. The tested hive version is 2.3.9. - -## Key features +## Key Features - [x] [batch](../../concept/connector-v2-features.md) - -Currently, only supports hudi cow table and Snapshot Query with Batch Mode - - [ ] [stream](../../concept/connector-v2-features.md) - [x] [exactly-once](../../concept/connector-v2-features.md) - [ ] [column projection](../../concept/connector-v2-features.md) - [x] [parallelism](../../concept/connector-v2-features.md) - [ ] [support user-defined split](../../concept/connector-v2-features.md) -## Options - -| name | type | required | default value | -|-------------------------|---------|------------------------------|---------------| -| table.path | string | yes | - | -| table.type | string | yes | - | -| conf.files | string | yes | - | -| use.kerberos | boolean | no | false | -| kerberos.principal | string | yes when use.kerberos = true | - | -| kerberos.principal.file | string | yes when use.kerberos = true | - | -| common-options | config | no | - | - -### table.path [string] - -`table.path` The hdfs root path of hudi table,such as 'hdfs://nameserivce/data/hudi/hudi_table/'. +## Description -### table.type [string] +Used to read data from Hudi. Currently, only supports hudi cow table and Snapshot Query with Batch Mode. -`table.type` The type of hudi table. Now we only support 'cow', 'mor' is not support yet. +In order to use this connector, You must ensure your spark/flink cluster already integrated hive. The tested hive version is 2.3.9. -### conf.files [string] +## Supported DataSource Info -`conf.files` The environment conf file path list(local path), which used to init hdfs client to read hudi table file. The example is '/home/test/hdfs-site.xml;/home/test/core-site.xml;/home/test/yarn-site.xml'. +:::tip -### use.kerberos [boolean] +* Currently, only supports Hudi cow table and Snapshot Query with Batch Mode -`use.kerberos` Whether to enable Kerberos, default is false. +::: -### kerberos.principal [string] +## Data Type Mapping -`kerberos.principal` When use kerberos, we should set kerberos princal such as 'test_user@xxx'. +| Hudi Data type | Seatunnel Data type | +|----------------|---------------------| +| ALL TYPE | STRING | -### kerberos.principal.file [string] +## Source Options -`kerberos.principal.file` When use kerberos, we should set kerberos princal file such as '/home/test/test_user.keytab'. +| Name | Type | Required | Default | Description | +|-------------------------|--------|------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| table.path | String | Yes | - | The hdfs root path of hudi table,such as 'hdfs://nameserivce/data/hudi/hudi_table/'. | +| table.type | String | Yes | - | The type of hudi table. Now we only support 'cow', 'mor' is not support yet. | +| conf.files | String | Yes | - | The environment conf file path list(local path), which used to init hdfs client to read hudi table file. The example is '/home/test/hdfs-site.xml;/home/test/core-site.xml;/home/test/yarn-site.xml'. | +| use.kerberos | bool | No | false | Whether to enable Kerberos, default is false. | +| kerberos.principal | String | yes when use.kerberos = true | - | When use kerberos, we should set kerberos principal such as 'test_user@xxx'. | +| kerberos.principal.file | string | yes when use.kerberos = true | - | When use kerberos, we should set kerberos principal file such as '/home/test/test_user.keytab'. | +| common-options | config | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. | -### common options +## Task Example -Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. +### Simple: -## Examples +> This example reads from a Hudi COW table and configures Kerberos for the environment, printing to the console. ```hocon -source { - +# Defining the runtime environment +env { + # You can set flink configuration here + execution.parallelism = 2 + job.mode = "BATCH" +} +source{ Hudi { table.path = "hdfs://nameserivce/data/hudi/hudi_table/" table.type = "cow" @@ -73,7 +71,15 @@ source { kerberos.principal = "test_user@xxx" kerberos.principal.file = "/home/test/test_user.keytab" } +} + +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/transform-v2/sql/ +} +sink { + Console {} } ``` From b62b6741ef584d1e284abac0984198fc691c56c9 Mon Sep 17 00:00:00 2001 From: Carl-Zhou-CN <67902676+Carl-Zhou-CN@users.noreply.github.com> Date: Fri, 28 Jul 2023 15:11:55 +0800 Subject: [PATCH 27/43] [Docs][Connector-V2][Doris] Reconstruct the Doris connector document (#4903) * [Docs][Connector-V2][Doris] Reconstruct the Doris connector document --------- Co-authored-by: zhouyao --- docs/en/connector-v2/sink/Doris.md | 226 +++++++++++++++++++++++------ 1 file changed, 179 insertions(+), 47 deletions(-) diff --git a/docs/en/connector-v2/sink/Doris.md b/docs/en/connector-v2/sink/Doris.md index f586ac3bcca0..506cb7f2485c 100644 --- a/docs/en/connector-v2/sink/Doris.md +++ b/docs/en/connector-v2/sink/Doris.md @@ -2,11 +2,24 @@ > Doris sink connector +## Support Those Engines + +> Spark
+> Flink
+> SeaTunnel Zeta
+ +## Key Features + +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [x] [cdc](../../concept/connector-v2-features.md) + ## Description Used to send data to Doris. Both support streaming and batch mode. The internal implementation of Doris sink connector is cached and imported by stream load in batches. +## Supported DataSource Info + :::tip Version Supported @@ -17,67 +30,186 @@ Version Supported ::: -## Key features - -- [x] [exactly-once](../../concept/connector-v2-features.md) -- [x] [cdc](../../concept/connector-v2-features.md) - -## Options - -| name | type | required | default value | -|--------------------|--------|----------|---------------| -| fenodes | string | yes | - | -| username | string | yes | - | -| password | string | yes | - | -| table.identifier | string | yes | - | -| sink.label-prefix | string | yes | - | -| sink.enable-2pc | bool | no | true | -| sink.enable-delete | bool | no | false | -| doris.config | map | yes | - | - -### fenodes [string] - -`Doris` cluster fenodes address, the format is `"fe_ip:fe_http_port, ..."` - -### username [string] - -`Doris` user username - -### password [string] - -`Doris` user password +## Sink Options + +| Name | Type | Required | Default | Description | +|---------------------|--------|----------|------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| fenodes | String | Yes | - | `Doris` cluster fenodes address, the format is `"fe_ip:fe_http_port, ..."` | +| username | String | Yes | - | `Doris` user username | +| password | String | Yes | - | `Doris` user password | +| table.identifier | String | Yes | - | The name of `Doris` table | +| sink.label-prefix | String | Yes | - | The label prefix used by stream load imports. In the 2pc scenario, global uniqueness is required to ensure the EOS semantics of SeaTunnel. | +| sink.enable-2pc | bool | No | - | Whether to enable two-phase commit (2pc), the default is true, to ensure Exactly-Once semantics. For two-phase commit, please refer to [here](https://doris.apache.org/docs/dev/sql-manual/sql-reference/Data-Manipulation-Statements/Load/STREAM-LOAD). | +| sink.enable-delete | bool | No | - | Whether to enable deletion. This option requires Doris table to enable batch delete function (0.15+ version is enabled by default), and only supports Unique model. you can get more detail at this [link](https://doris.apache.org/docs/dev/data-operate/update-delete/batch-delete-manual) | +| sink.check-interval | int | No | 10000 | check exception with the interval while loading | +| sink.max-retries | int | No | 3 | the max retry times if writing records to database failed | +| sink.buffer-size | int | No | 256 * 1024 | the buffer size to cache data for stream load. | +| sink.buffer-count | int | No | 3 | the buffer count to cache data for stream load. | +| doris.config | map | yes | - | This option is used to support operations such as `insert`, `delete`, and `update` when automatically generate sql,and supported formats. | + +## Data Type Mapping + +| Doris Data type | SeaTunnel Data type | +|-----------------|-----------------------------------------| +| BOOLEAN | BOOLEAN | +| TINYINT | TINYINT | +| SMALLINT | SMALLINT
TINYINT | +| INT | INT
SMALLINT
TINYINT | +| BIGINT | BIGINT
INT
SMALLINT
TINYINT | +| LARGEINT | BIGINT
INT
SMALLINT
TINYINT | +| FLOAT | FLOAT | +| DOUBLE | DOUBLE
FLOAT | +| DECIMAL | DECIMAL
DOUBLE
FLOAT | +| DATE | DATE | +| DATETIME | TIMESTAMP | +| CHAR | STRING | +| VARCHAR | STRING | +| STRING | STRING | +| ARRAY | ARRAY | +| MAP | MAP | +| JSON | STRING | +| HLL | Not supported yet | +| BITMAP | Not supported yet | +| QUANTILE_STATE | Not supported yet | +| STRUCT | Not supported yet | -### table.identifier [string] - -The name of `Doris` table +#### Supported import data formats -### sink.label-prefix [string] +The supported formats include CSV and JSON -The label prefix used by stream load imports. In the 2pc scenario, global uniqueness is required to ensure the EOS semantics of SeaTunnel. +## Task Example -### sink.enable-2pc [bool] +### Simple: -Whether to enable two-phase commit (2pc), the default is true, to ensure Exactly-Once semantics. For two-phase commit, please refer to [here](https://doris.apache.org/docs/dev/sql-manual/sql-reference/Data-Manipulation-Statements/Load/STREAM-LOAD). +> The following example describes writing multiple data types to Doris, and users need to create corresponding tables downstream -### sink.enable-delete [bool] +```hocon +env { + parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 10000 +} -Whether to enable deletion. This option requires Doris table to enable batch delete function (0.15+ version is enabled by default), and only supports Unique model. you can get more detail at this link: +source { + FakeSource { + row.num = 10 + map.size = 10 + array.size = 10 + bytes.length = 10 + string.length = 10 + schema = { + fields { + c_map = "map>" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_decimal = "decimal(16, 1)" + c_null = "null" + c_bytes = bytes + c_date = date + c_timestamp = timestamp + } + } + } +} -https://doris.apache.org/docs/dev/data-operate/update-delete/batch-delete-manual +sink { + Doris { + fenodes = "doris_cdc_e2e:8030" + username = root + password = "" + table.identifier = "test.e2e_table_sink" + sink.label-prefix = "test-cdc" + sink.enable-2pc = "true" + sink.enable-delete = "true" + doris.config { + format = "json" + read_json_by_line = "true" + } + } +} +``` -### doris.config [map] +### CDC(Change Data Capture) Event: -The parameter of the stream load `data_desc`, you can get more detail at this link: +> This example defines a SeaTunnel synchronization task that automatically generates data through FakeSource and sends it to Doris Sink,FakeSource simulates CDC data with schema, score (int type),Doris needs to create a table sink named test.e2e_table_sink and a corresponding table for it. -https://doris.apache.org/docs/dev/sql-manual/sql-reference/Data-Manipulation-Statements/Load/STREAM-LOAD +```hocon +env { + parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 10000 +} -#### Supported import data formats +source { + FakeSource { + schema = { + fields { + pk_id = bigint + name = string + score = int + sex = boolean + number = tinyint + height = float + sight = double + create_time = date + update_time = timestamp + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100, true, 1, 170.0, 4.3, "2020-02-02", "2020-02-02T02:02:02"] + }, + { + kind = INSERT + fields = [2, "B", 100, true, 1, 170.0, 4.3, "2020-02-02", "2020-02-02T02:02:02"] + }, + { + kind = INSERT + fields = [3, "C", 100, true, 1, 170.0, 4.3, "2020-02-02", "2020-02-02T02:02:02"] + }, + { + kind = UPDATE_BEFORE + fields = [1, "A", 100, true, 1, 170.0, 4.3, "2020-02-02", "2020-02-02T02:02:02"] + }, + { + kind = UPDATE_AFTER + fields = [1, "A_1", 100, true, 1, 170.0, 4.3, "2020-02-02", "2020-02-02T02:02:02"] + }, + { + kind = DELETE + fields = [2, "B", 100, true, 1, 170.0, 4.3, "2020-02-02", "2020-02-02T02:02:02"] + } + ] + } +} -The supported formats include CSV and JSON. Default value: CSV +sink { + Doris { + fenodes = "doris_cdc_e2e:8030" + username = root + password = "" + table.identifier = "test.e2e_table_sink" + sink.label-prefix = "test-cdc" + sink.enable-2pc = "true" + sink.enable-delete = "true" + doris.config { + format = "json" + read_json_by_line = "true" + } + } +} -## Example +``` -Use JSON format to import data +### Use JSON format to import data ``` sink { @@ -97,7 +229,7 @@ sink { ``` -Use CSV format to import data +### Use CSV format to import data ``` sink { From 94fd6755e6a97121ba5706e4d6b15afe4d432a21 Mon Sep 17 00:00:00 2001 From: ic4y <83933160+ic4y@users.noreply.github.com> Date: Sat, 29 Jul 2023 11:58:56 +0800 Subject: [PATCH 28/43] [improve] [CDC Base] Add some split parameters to the optionRule (#5161) --- .../cdc/mysql/source/MySqlIncrementalSourceFactory.java | 5 ++++- .../source/source/SqlServerIncrementalSourceFactory.java | 5 ++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlIncrementalSourceFactory.java index 7e64ee81ef7a..396fd7bae9d4 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlIncrementalSourceFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mysql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/MySqlIncrementalSourceFactory.java @@ -65,7 +65,10 @@ public OptionRule optionRule() { JdbcSourceOptions.SERVER_TIME_ZONE, JdbcSourceOptions.CONNECT_TIMEOUT_MS, JdbcSourceOptions.CONNECT_MAX_RETRIES, - JdbcSourceOptions.CONNECTION_POOL_SIZE) + JdbcSourceOptions.CONNECTION_POOL_SIZE, + JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND, + JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND, + JdbcSourceOptions.SAMPLE_SHARDING_THRESHOLD) .optional(MySqlSourceOptions.STARTUP_MODE, MySqlSourceOptions.STOP_MODE) .conditional( MySqlSourceOptions.STARTUP_MODE, diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerIncrementalSourceFactory.java index 85810b825f30..285d4b792322 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerIncrementalSourceFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-sqlserver/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/sqlserver/source/source/SqlServerIncrementalSourceFactory.java @@ -65,7 +65,10 @@ public OptionRule optionRule() { JdbcSourceOptions.SERVER_TIME_ZONE, JdbcSourceOptions.CONNECT_TIMEOUT_MS, JdbcSourceOptions.CONNECT_MAX_RETRIES, - JdbcSourceOptions.CONNECTION_POOL_SIZE) + JdbcSourceOptions.CONNECTION_POOL_SIZE, + JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND, + JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND, + JdbcSourceOptions.SAMPLE_SHARDING_THRESHOLD) .optional(SqlServerSourceOptions.STARTUP_MODE, SqlServerSourceOptions.STOP_MODE) .conditional( SqlServerSourceOptions.STARTUP_MODE, From 2a85525f4ca783ba2762672ec94e44749c4fd088 Mon Sep 17 00:00:00 2001 From: ic4y <83933160+ic4y@users.noreply.github.com> Date: Sun, 30 Jul 2023 14:04:41 +0800 Subject: [PATCH 29/43] [bugfix] [File Base] Fix Hadoop Kerberos authentication related issues. (#5171) --- .../seatunnel/file/source/reader/AbstractReadStrategy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java index 54e70662425d..ea6c902c05a9 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java @@ -72,7 +72,7 @@ public abstract class AbstractReadStrategy implements ReadStrategy { protected List readColumns = new ArrayList<>(); protected boolean isMergePartition = true; protected long skipHeaderNumber = BaseSourceConfig.SKIP_HEADER_ROW_NUMBER.defaultValue(); - protected boolean isKerberosAuthorization = false; + protected transient boolean isKerberosAuthorization = false; @Override public void init(HadoopConf conf) { From e4f666f7d1a506ce85ec1834e35ddc2009ad0587 Mon Sep 17 00:00:00 2001 From: Tyrantlucifer Date: Mon, 31 Jul 2023 10:05:23 +0800 Subject: [PATCH 30/43] [CI] add code style check when docs changed (#5183) --- .github/workflows/documents.yml | 11 +++++++++++ docs/en/connector-v2/sink/Doris.md | 4 ++-- docs/en/connector-v2/source/Hudi.md | 2 +- 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/.github/workflows/documents.yml b/.github/workflows/documents.yml index 3a97a3a7de28..7c3c56cf07b7 100644 --- a/.github/workflows/documents.yml +++ b/.github/workflows/documents.yml @@ -52,3 +52,14 @@ jobs: cd seatunnel-website npm install npm run build + + code-style: + name: Code style + runs-on: ubuntu-latest + timeout-minutes: 10 + steps: + - uses: actions/checkout@v3 + with: + submodules: true + - name: Check code style + run: ./mvnw --batch-mode --quiet --no-snapshot-updates clean spotless:check diff --git a/docs/en/connector-v2/sink/Doris.md b/docs/en/connector-v2/sink/Doris.md index 506cb7f2485c..6bf8dc5369c9 100644 --- a/docs/en/connector-v2/sink/Doris.md +++ b/docs/en/connector-v2/sink/Doris.md @@ -32,7 +32,7 @@ Version Supported ## Sink Options -| Name | Type | Required | Default | Description | +| Name | Type | Required | Default | Description | |---------------------|--------|----------|------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | fenodes | String | Yes | - | `Doris` cluster fenodes address, the format is `"fe_ip:fe_http_port, ..."` | | username | String | Yes | - | `Doris` user username | @@ -49,7 +49,7 @@ Version Supported ## Data Type Mapping -| Doris Data type | SeaTunnel Data type | +| Doris Data type | SeaTunnel Data type | |-----------------|-----------------------------------------| | BOOLEAN | BOOLEAN | | TINYINT | TINYINT | diff --git a/docs/en/connector-v2/source/Hudi.md b/docs/en/connector-v2/source/Hudi.md index b70d34608ea0..ffe17f7de715 100644 --- a/docs/en/connector-v2/source/Hudi.md +++ b/docs/en/connector-v2/source/Hudi.md @@ -39,7 +39,7 @@ In order to use this connector, You must ensure your spark/flink cluster already ## Source Options -| Name | Type | Required | Default | Description | +| Name | Type | Required | Default | Description | |-------------------------|--------|------------------------------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | table.path | String | Yes | - | The hdfs root path of hudi table,such as 'hdfs://nameserivce/data/hudi/hudi_table/'. | | table.type | String | Yes | - | The type of hudi table. Now we only support 'cow', 'mor' is not support yet. | From 1db9f45d39fee3f2a67ffafe719d97ecdfbb7a1f Mon Sep 17 00:00:00 2001 From: Chengyu Yan Date: Mon, 31 Jul 2023 11:22:34 +0800 Subject: [PATCH 31/43] [Bug][Translation][Spark] Fix SeaTunnelRowConvertor fail to convert when schema contains row type. (#5170) --- release-note.md | 2 ++ .../execution/TransformExecuteProcessor.java | 1 - .../src/test/resources/copy_transform.conf | 6 ++++ .../filter_row_kind_exclude_delete.conf | 5 ++++ .../filter_row_kind_exclude_insert.conf | 5 ++++ .../filter_row_kind_include_insert.conf | 5 ++++ .../src/test/resources/filter_transform.conf | 7 ++++- .../src/test/resources/split_transform.conf | 5 ++++ .../resources/field_mapper_transform.conf | 6 ++++ .../src/test/resources/sql_transform.conf | 7 ++++- .../serialization/SeaTunnelRowConverter.java | 29 ++++++++++++++++--- 11 files changed, 71 insertions(+), 7 deletions(-) diff --git a/release-note.md b/release-note.md index 0e84da433c25..b542b35a8148 100644 --- a/release-note.md +++ b/release-note.md @@ -7,6 +7,8 @@ - [Core] [API] Fix parse nested row data type key changed upper (#4459) - [Starter][Flink]Support transform-v2 for flink #3396 - [Flink] Support flink 1.14.x #3963 +- [Core][Translation][Spark] Fix SeaTunnelRowConvertor fail to convert when schema contains row type (#5170) + ### Transformer - [Spark] Support transform-v2 for spark (#3409) - [ALL]Add FieldMapper Transform #3781 diff --git a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java index 179598b3a612..fc9be559257f 100644 --- a/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java +++ b/seatunnel-core/seatunnel-spark-starter/seatunnel-spark-starter-common/src/main/java/org/apache/seatunnel/core/starter/spark/execution/TransformExecuteProcessor.java @@ -185,7 +185,6 @@ public Row next() { return null; } seaTunnelRow = outputRowConverter.convert(seaTunnelRow); - return new GenericRowWithSchema(seaTunnelRow.getFields(), structType); } catch (Exception e) { throw new TaskExecuteException("Row convert failed, caused: " + e.getMessage(), e); diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/copy_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/copy_transform.conf index 25ca4ce5f9ae..b937b0a8cbe8 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/copy_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/copy_transform.conf @@ -30,6 +30,11 @@ source { fields { id = "int" name = "string" + c_row = { + c_row = { + c_int = int + } + } } } } @@ -49,6 +54,7 @@ transform { id_1 = "id" name2 = "name" name3 = "name" + c_row_1 = "c_row" } } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_exclude_delete.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_exclude_delete.conf index f7fc0f6e0e11..8fdf195b037c 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_exclude_delete.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_exclude_delete.conf @@ -31,6 +31,11 @@ source { id = "int" name = "string" age = "int" + c_row = { + c_row = { + c_int = int + } + } } } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_exclude_insert.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_exclude_insert.conf index cc36417788ba..9fc0e577cb8f 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_exclude_insert.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_exclude_insert.conf @@ -31,6 +31,11 @@ source { id = "int" name = "string" age = "int" + c_row = { + c_row = { + c_int = int + } + } } } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_include_insert.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_include_insert.conf index d1fbf79bea21..72d1e38cd442 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_include_insert.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_row_kind_include_insert.conf @@ -31,6 +31,11 @@ source { id = "int" name = "string" age = "int" + c_row = { + c_row = { + c_int = int + } + } } } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_transform.conf index 56439b4414f6..c869c70a77b7 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/filter_transform.conf @@ -31,6 +31,11 @@ source { id = "int" name = "string" age = "int" + c_row = { + c_row = { + c_int = int + } + } } } } @@ -40,7 +45,7 @@ transform { Filter { source_table_name = "fake" result_table_name = "fake1" - fields = ["age", "name"] + fields = ["age", "name", "c_row"] } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/split_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/split_transform.conf index 61e10f694ac7..7ad9fbf8f4af 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/split_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-1/src/test/resources/split_transform.conf @@ -31,6 +31,11 @@ source { id = "int" name = "string" age = "int" + c_row = { + c_row = { + c_int = int + } + } } } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/field_mapper_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/field_mapper_transform.conf index c2d1f225f2b9..59d19f3ee74e 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/field_mapper_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/field_mapper_transform.conf @@ -34,6 +34,11 @@ source { string1 = "string" int1 = "int" c_bigint = "bigint" + c_row = { + c_row = { + c_int = int + } + } } } } @@ -48,6 +53,7 @@ transform { age = age_as int1 = int1_as name = name + c_row = c_row } } } diff --git a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform.conf b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform.conf index c5f7c4047e74..78e21280f0de 100644 --- a/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform.conf +++ b/seatunnel-e2e/seatunnel-transforms-v2-e2e/seatunnel-transforms-v2-e2e-part-2/src/test/resources/sql_transform.conf @@ -36,6 +36,11 @@ source { c_map = "map" c_array = "array" c_decimal = "decimal(30, 8)" + c_row = { + c_row = { + c_int = int + } + } } } } @@ -46,7 +51,7 @@ transform { source_table_name = "fake" result_table_name = "fake1" # the query table name must same as field 'source_table_name' - query = "select id, regexp_replace(name, '.+', 'b') as name, age+1 as age, pi() as pi, c_timestamp, c_date, c_map, c_array, c_decimal from fake" + query = "select id, regexp_replace(name, '.+', 'b') as name, age+1 as age, pi() as pi, c_timestamp, c_date, c_map, c_array, c_decimal, c_row from fake" } # The SQL transform support base function and criteria operation # But the complex SQL unsupported yet, include: multi source table/rows JOIN and AGGREGATE operation and the like diff --git a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java index 51d5c7308bd5..15357204cd3e 100644 --- a/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java +++ b/seatunnel-translation/seatunnel-translation-spark/seatunnel-translation-spark-common/src/main/java/org/apache/seatunnel/translation/spark/serialization/SeaTunnelRowConverter.java @@ -24,7 +24,10 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.translation.serialization.RowConverter; +import org.apache.seatunnel.translation.spark.utils.TypeConverterUtils; +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; +import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; import scala.Tuple2; @@ -51,7 +54,11 @@ public SeaTunnelRowConverter(SeaTunnelDataType dataType) { @Override public SeaTunnelRow convert(SeaTunnelRow seaTunnelRow) throws IOException { validate(seaTunnelRow); - return (SeaTunnelRow) convert(seaTunnelRow, dataType); + GenericRowWithSchema rowWithSchema = (GenericRowWithSchema) convert(seaTunnelRow, dataType); + SeaTunnelRow newRow = new SeaTunnelRow(rowWithSchema.values()); + newRow.setRowKind(seaTunnelRow.getRowKind()); + newRow.setTableId(seaTunnelRow.getTableId()); + return newRow; } private Object convert(Object field, SeaTunnelDataType dataType) { @@ -62,7 +69,7 @@ private Object convert(Object field, SeaTunnelDataType dataType) { case ROW: SeaTunnelRow seaTunnelRow = (SeaTunnelRow) field; SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; - return convert(seaTunnelRow, rowType); + return convertRow(seaTunnelRow, rowType); case DATE: return Date.valueOf((LocalDate) field); case TIMESTAMP: @@ -94,16 +101,17 @@ private Object convert(Object field, SeaTunnelDataType dataType) { } } - private SeaTunnelRow convert(SeaTunnelRow seaTunnelRow, SeaTunnelRowType rowType) { + private GenericRowWithSchema convertRow(SeaTunnelRow seaTunnelRow, SeaTunnelRowType rowType) { int arity = rowType.getTotalFields(); Object[] values = new Object[arity]; + StructType schema = (StructType) TypeConverterUtils.convert(rowType); for (int i = 0; i < arity; i++) { Object fieldValue = convert(seaTunnelRow.getField(i), rowType.getFieldType(i)); if (fieldValue != null) { values[i] = fieldValue; } } - return new SeaTunnelRow(values); + return new GenericRowWithSchema(values, schema); } private scala.collection.immutable.HashMap convertMap( @@ -148,6 +156,10 @@ private Object reconvert(Object field, SeaTunnelDataType dataType) { } switch (dataType.getSqlType()) { case ROW: + if (field instanceof GenericRowWithSchema) { + return createFromGenericRow( + (GenericRowWithSchema) field, (SeaTunnelRowType) dataType); + } return reconvert((SeaTunnelRow) field, (SeaTunnelRowType) dataType); case DATE: return ((Date) field).toLocalDate(); @@ -166,6 +178,15 @@ private Object reconvert(Object field, SeaTunnelDataType dataType) { } } + private SeaTunnelRow createFromGenericRow(GenericRowWithSchema row, SeaTunnelRowType type) { + Object[] fields = row.values(); + Object[] newFields = new Object[fields.length]; + for (int idx = 0; idx < fields.length; idx++) { + newFields[idx] = reconvert(fields[idx], type.getFieldType(idx)); + } + return new SeaTunnelRow(newFields); + } + private SeaTunnelRow reconvert(SeaTunnelRow engineRow, SeaTunnelRowType rowType) { int num = engineRow.getFields().length; Object[] fields = new Object[num]; From bc28eb1747d880babe7e04b8b399caf20da8f74b Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Mon, 31 Jul 2023 14:50:38 +0800 Subject: [PATCH 32/43] [Improve][Zeta] Move checkpoint notify complete in checkpoint stage (#5185) --- .../server/checkpoint/CheckpointCoordinator.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java index 0f9e03df349c..e6b3c3d27ac7 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/checkpoint/CheckpointCoordinator.java @@ -695,14 +695,6 @@ public synchronized void completePendingCheckpoint(CompletedCheckpoint completed completedCheckpoint.getCheckpointTimestamp(), completedCheckpoint.getCompletedTimestamp()); final long checkpointId = completedCheckpoint.getCheckpointId(); - pendingCheckpoints.remove(checkpointId); - pendingCounter.decrementAndGet(); - if (pendingCheckpoints.size() + 1 == coordinatorConfig.getMaxConcurrentCheckpoints()) { - // latest checkpoint completed time > checkpoint interval - if (notFinalCheckpoint(completedCheckpoint.getCheckpointType())) { - scheduleTriggerPendingCheckpoint(0L); - } - } completedCheckpoints.addLast(completedCheckpoint); try { byte[] states = serializer.serialize(completedCheckpoint); @@ -742,6 +734,14 @@ public synchronized void completePendingCheckpoint(CompletedCheckpoint completed completedCheckpoint.getJobId()); latestCompletedCheckpoint = completedCheckpoint; notifyCompleted(completedCheckpoint); + pendingCheckpoints.remove(checkpointId); + pendingCounter.decrementAndGet(); + if (pendingCheckpoints.size() + 1 == coordinatorConfig.getMaxConcurrentCheckpoints()) { + // latest checkpoint completed time > checkpoint interval + if (notFinalCheckpoint(completedCheckpoint.getCheckpointType())) { + scheduleTriggerPendingCheckpoint(0L); + } + } if (isCompleted()) { cleanPendingCheckpoint(CheckpointCloseReason.CHECKPOINT_COORDINATOR_COMPLETED); if (latestCompletedCheckpoint.getCheckpointType().equals(SAVEPOINT_TYPE)) { From 63eb1376718ce96afd04d1a95edc0f8be5f945cf Mon Sep 17 00:00:00 2001 From: XiaoJiang521 <131635688+XiaoJiang521@users.noreply.github.com> Date: Mon, 31 Jul 2023 16:41:04 +0800 Subject: [PATCH 33/43] [Feature][Catalog] Add JDBC Catalog auto create table (#4917) --- .github/workflows/backend.yml | 24 + .../api/table/catalog/CatalogTable.java | 27 + .../seatunnel/api/table/catalog/Column.java | 53 ++ .../api/table/catalog/PhysicalColumn.java | 86 +++- .../jdbc/catalog/AbstractJdbcCatalog.java | 19 +- .../jdbc/catalog/JdbcCatalogOptions.java | 23 +- .../jdbc/catalog/mysql/MySqlCatalog.java | 138 +++-- .../catalog/mysql/MySqlCatalogFactory.java | 4 +- .../mysql/MysqlCreateTableSqlBuilder.java | 94 +++- .../catalog/mysql/MysqlDataTypeConvertor.java | 8 +- .../jdbc/catalog/oracle/OracleCatalog.java | 361 +++++++++++++ .../catalog/oracle/OracleCatalogFactory.java | 61 +++ .../oracle/OracleCreateTableSqlBuilder.java | 170 ++++++ .../oracle/OracleDataTypeConvertor.java | 200 ++++++++ .../jdbc/catalog/oracle/OracleURLParser.java | 43 ++ .../jdbc/catalog/psql/PostgresCatalog.java | 441 ++++++++++++++++ .../catalog/psql/PostgresCatalogFactory.java | 62 +++ .../psql/PostgresCreateTableSqlBuilder.java | 144 ++++++ .../psql/PostgresDataTypeConvertor.java | 81 +-- .../catalog/sqlserver/SqlServerCatalog.java | 172 +++++-- .../sqlserver/SqlServerCatalogFactory.java | 6 +- .../SqlServerCreateTableSqlBuilder.java | 310 +++++++++++ .../sqlserver/SqlServerDataTypeConvertor.java | 4 +- .../catalog/sqlserver/SqlServerURLParser.java | 8 +- .../seatunnel/jdbc/config/JdbcOptions.java | 19 +- .../seatunnel/jdbc/config/JdbcSinkConfig.java | 18 +- .../internal/JdbcOutputFormatBuilder.java | 80 ++- .../jdbc/internal/dialect/JdbcDialect.java | 21 +- .../internal/dialect/mysql/MysqlDialect.java | 6 + .../sqlserver/SqlserverJdbcRowConverter.java | 88 ++++ .../seatunnel/jdbc/sink/JdbcSink.java | 47 +- .../seatunnel/jdbc/sink/JdbcSinkFactory.java | 48 +- .../jdbc/catalog/mysql/MySqlCatalogTest.java | 124 +++++ .../catalog/oracle/OracleCatalogTest.java | 65 +++ .../catalog/psql/PostgresCatalogTest.java | 59 +++ .../sql/MysqlCreateTableSqlBuilderTest.java | 17 +- .../sqlserver/SqlServerCatalogTest.java | 132 +++++ .../sqlserver/SqlServerURLParserTest.java | 42 ++ .../test/resources/mongodbcdc_to_mysql.conf | 2 +- .../src/test/resources/mysqlcdc_to_mysql.conf | 2 +- .../resources/sqlservercdc_to_console.conf | 4 +- .../connector-jdbc-e2e-part-3/pom.xml | 22 + .../jdbc/JdbcSinkCDCChangelogIT.java | 10 + .../jdbc_postgres_source_and_sink.conf | 2 +- .../resources/jdbc_sink_cdc_changelog.conf | 2 +- .../connector-jdbc-e2e-part-4/pom.xml | 96 ++++ .../jdbc/JdbcMySqlCreateTableIT.java | 471 +++++++++++++++++ .../jdbc/JdbcSqlServerCreateTableIT.java | 482 ++++++++++++++++++ .../connector-jdbc-e2e/pom.xml | 1 + .../e2e/connector/kafka/CanalToKafkaIT.java | 11 + .../kafka_source_canal_cdc_to_pgsql.conf | 5 +- .../e2e/connector/pulsar/CanalToPulsarIT.java | 11 + .../resources/cdc_canal_pulsar_to_pg.conf | 4 +- 53 files changed, 4221 insertions(+), 209 deletions(-) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilder.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleDataTypeConvertor.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleURLParser.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogFactory.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilder.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilder.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerURLParserTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java diff --git a/.github/workflows/backend.yml b/.github/workflows/backend.yml index 6da4f4a5ab64..13a4d4b52d9d 100644 --- a/.github/workflows/backend.yml +++ b/.github/workflows/backend.yml @@ -736,6 +736,30 @@ jobs: env: MAVEN_OPTS: -Xmx4096m + jdbc-connectors-it-part-4: + needs: [ changes, sanity-check ] + if: needs.changes.outputs.api == 'true' + runs-on: ${{ matrix.os }} + strategy: + matrix: + java: [ '8', '11' ] + os: [ 'ubuntu-latest' ] + timeout-minutes: 90 + steps: + - uses: actions/checkout@v2 + - name: Set up JDK ${{ matrix.java }} + uses: actions/setup-java@v3 + with: + java-version: ${{ matrix.java }} + distribution: 'temurin' + cache: 'maven' + - name: run jdbc connectors integration test (part-4) + if: needs.changes.outputs.api == 'true' + run: | + ./mvnw -B -T 1C verify -DskipUT=true -DskipIT=false -D"license.skipAddThirdParty"=true --no-snapshot-updates -pl :connector-jdbc-e2e-part-4 -am -Pci + env: + MAVEN_OPTS: -Xmx4096m + kafka-connector-it: needs: [ changes, sanity-check ] if: needs.changes.outputs.api == 'true' diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTable.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTable.java index 3aa50335910d..1be6de028417 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTable.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/CatalogTable.java @@ -38,6 +38,8 @@ public final class CatalogTable implements Serializable { private final String comment; + private final String catalogName; + public static CatalogTable of( TableIdentifier tableId, TableSchema tableSchema, @@ -47,17 +49,38 @@ public static CatalogTable of( return new CatalogTable(tableId, tableSchema, options, partitionKeys, comment); } + public static CatalogTable of( + TableIdentifier tableId, + TableSchema tableSchema, + Map options, + List partitionKeys, + String comment, + String catalogName) { + return new CatalogTable(tableId, tableSchema, options, partitionKeys, comment, catalogName); + } + private CatalogTable( TableIdentifier tableId, TableSchema tableSchema, Map options, List partitionKeys, String comment) { + this(tableId, tableSchema, options, partitionKeys, comment, ""); + } + + private CatalogTable( + TableIdentifier tableId, + TableSchema tableSchema, + Map options, + List partitionKeys, + String comment, + String catalogName) { this.tableId = tableId; this.tableSchema = tableSchema; this.options = options; this.partitionKeys = partitionKeys; this.comment = comment; + this.catalogName = catalogName; } public TableIdentifier getTableId() { @@ -80,6 +103,10 @@ public String getComment() { return comment; } + public String getCatalogName() { + return catalogName; + } + @Override public String toString() { return "CatalogTable{" diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/Column.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/Column.java index b528996a3aec..bec10b3d7581 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/Column.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/Column.java @@ -23,6 +23,7 @@ import lombok.Data; import java.io.Serializable; +import java.util.Map; /** * Represent the column of {@link TableSchema}. @@ -54,6 +55,24 @@ public abstract class Column implements Serializable { protected final String comment; + /** Field type in the database * */ + protected final String sourceType; + + /** Unsigned bit * */ + protected final boolean isUnsigned; + + /** Whether to use the 0 bit * */ + protected final boolean isZeroFill; + + /** Bit length * */ + protected final Long bitLen; + + /** integer may be cross the border * */ + protected final Long longColumnLength; + + /** your options * */ + protected final Map options; + protected Column( String name, SeaTunnelDataType dataType, @@ -61,12 +80,46 @@ protected Column( boolean nullable, Object defaultValue, String comment) { + this( + name, + dataType, + columnLength, + nullable, + defaultValue, + comment, + null, + false, + false, + null, + 0L, + null); + } + + protected Column( + String name, + SeaTunnelDataType dataType, + Integer columnLength, + boolean nullable, + Object defaultValue, + String comment, + String sourceType, + boolean isUnsigned, + boolean isZeroFill, + Long bitLen, + Long longColumnLength, + Map options) { this.name = name; this.dataType = dataType; this.columnLength = columnLength; this.nullable = nullable; this.defaultValue = defaultValue; this.comment = comment; + this.sourceType = sourceType; + this.isUnsigned = isUnsigned; + this.isZeroFill = isZeroFill; + this.bitLen = bitLen; + this.longColumnLength = longColumnLength; + this.options = options; } /** diff --git a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/PhysicalColumn.java b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/PhysicalColumn.java index bc379e355466..164752d46863 100644 --- a/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/PhysicalColumn.java +++ b/seatunnel-api/src/main/java/org/apache/seatunnel/api/table/catalog/PhysicalColumn.java @@ -23,6 +23,8 @@ import lombok.EqualsAndHashCode; import lombok.ToString; +import java.util.Map; + /** Representation of a physical column. */ @EqualsAndHashCode(callSuper = true) @ToString(callSuper = true) @@ -38,6 +40,34 @@ protected PhysicalColumn( super(name, dataType, columnLength, nullable, defaultValue, comment); } + protected PhysicalColumn( + String name, + SeaTunnelDataType dataType, + Integer columnLength, + boolean nullable, + Object defaultValue, + String comment, + String sourceType, + boolean isUnsigned, + boolean isZeroFill, + Long bitLen, + Long longColumnLength, + Map options) { + super( + name, + dataType, + columnLength, + nullable, + defaultValue, + comment, + sourceType, + isUnsigned, + isZeroFill, + bitLen, + longColumnLength, + options); + } + public static PhysicalColumn of( String name, SeaTunnelDataType dataType, @@ -48,6 +78,34 @@ public static PhysicalColumn of( return new PhysicalColumn(name, dataType, columnLength, nullable, defaultValue, comment); } + public static PhysicalColumn of( + String name, + SeaTunnelDataType dataType, + Integer columnLength, + boolean nullable, + Object defaultValue, + String comment, + String sourceType, + boolean isUnsigned, + boolean isZeroFill, + Long bitLen, + Map options, + Long longColumnLength) { + return new PhysicalColumn( + name, + dataType, + columnLength, + nullable, + defaultValue, + comment, + sourceType, + isUnsigned, + isZeroFill, + bitLen, + longColumnLength, + options); + } + @Override public boolean isPhysical() { return true; @@ -55,11 +113,35 @@ public boolean isPhysical() { @Override public Column copy(SeaTunnelDataType newType) { - return PhysicalColumn.of(name, newType, columnLength, nullable, defaultValue, comment); + return PhysicalColumn.of( + name, + newType, + columnLength, + nullable, + defaultValue, + comment, + sourceType, + isUnsigned, + isZeroFill, + bitLen, + options, + longColumnLength); } @Override public Column copy() { - return PhysicalColumn.of(name, dataType, columnLength, nullable, defaultValue, comment); + return PhysicalColumn.of( + name, + dataType, + columnLength, + nullable, + defaultValue, + comment, + sourceType, + isUnsigned, + isZeroFill, + bitLen, + options, + longColumnLength); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java index 247ecc651fee..ee108ad5e7fc 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/AbstractJdbcCatalog.java @@ -64,10 +64,16 @@ public abstract class AbstractJdbcCatalog implements Catalog { protected final String suffix; protected final String defaultUrl; + protected final Optional defaultSchema; + protected Connection defaultConnection; public AbstractJdbcCatalog( - String catalogName, String username, String pwd, JdbcUrlUtil.UrlInfo urlInfo) { + String catalogName, + String username, + String pwd, + JdbcUrlUtil.UrlInfo urlInfo, + String defaultSchema) { checkArgument(StringUtils.isNotBlank(username)); urlInfo.getDefaultDatabase() @@ -78,10 +84,10 @@ public AbstractJdbcCatalog( this.defaultDatabase = urlInfo.getDefaultDatabase().get(); this.username = username; this.pwd = pwd; - String baseUrl = urlInfo.getUrlWithoutDatabase(); - this.baseUrl = baseUrl.endsWith("/") ? baseUrl : baseUrl + "/"; + this.baseUrl = urlInfo.getUrlWithoutDatabase(); this.defaultUrl = urlInfo.getOrigin(); this.suffix = urlInfo.getSuffix(); + this.defaultSchema = Optional.ofNullable(defaultSchema); } @Override @@ -248,6 +254,13 @@ public void createTable(TablePath tablePath, CatalogTable table, boolean ignoreI if (!databaseExists(tablePath.getDatabaseName())) { throw new DatabaseNotExistException(catalogName, tablePath.getDatabaseName()); } + if (defaultSchema.isPresent()) { + tablePath = + new TablePath( + tablePath.getDatabaseName(), + defaultSchema.get(), + tablePath.getTableName()); + } if (!createTableInternal(tablePath, table) && !ignoreIfExists) { throw new TableAlreadyExistException(catalogName, tablePath); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java index 3a664079aa5f..712eefacb84b 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/JdbcCatalogOptions.java @@ -43,6 +43,27 @@ public interface JdbcCatalogOptions { .noDefaultValue() .withDescription("Password to use when connecting to the database server."); + Option SCHEMA = + Options.key("schema") + .stringType() + .noDefaultValue() + .withDescription( + "for databases that support the schema parameter, give it priority."); + OptionRule.Builder BASE_RULE = - OptionRule.builder().required(BASE_URL).required(USERNAME, PASSWORD); + OptionRule.builder().required(BASE_URL).required(USERNAME, PASSWORD).optional(SCHEMA); + + Option TABLE_PREFIX = + Options.key("tablePrefix") + .stringType() + .noDefaultValue() + .withDescription( + "The table prefix name added when the table is automatically created"); + + Option TABLE_SUFFIX = + Options.key("tableSuffix") + .stringType() + .noDefaultValue() + .withDescription( + "The table suffix name added when the table is automatically created"); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java index 463c7a8bf769..267a68f0eefc 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalog.java @@ -35,6 +35,7 @@ import com.mysql.cj.MysqlType; import com.mysql.cj.jdbc.result.ResultSetImpl; import com.mysql.cj.util.StringUtils; +import lombok.extern.slf4j.Slf4j; import java.sql.Connection; import java.sql.DatabaseMetaData; @@ -48,14 +49,18 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +@Slf4j public class MySqlCatalog extends AbstractJdbcCatalog { protected static final Set SYS_DATABASES = new HashSet<>(4); + private final String SELECT_COLUMNS = + "SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = '%s' AND TABLE_NAME ='%s'"; static { SYS_DATABASES.add("information_schema"); @@ -68,7 +73,7 @@ public class MySqlCatalog extends AbstractJdbcCatalog { public MySqlCatalog( String catalogName, String username, String pwd, JdbcUrlUtil.UrlInfo urlInfo) { - super(catalogName, username, pwd, urlInfo); + super(catalogName, username, pwd, urlInfo, null); this.connectionMap = new ConcurrentHashMap<>(); } @@ -127,7 +132,8 @@ public List listTables(String databaseName) } String dbUrl = getUrlFromDatabaseName(databaseName); - try (PreparedStatement ps = getConnection(dbUrl).prepareStatement("SHOW TABLES;")) { + Connection connection = getConnection(dbUrl); + try (PreparedStatement ps = connection.prepareStatement("SHOW TABLES;")) { ResultSet rs = ps.executeQuery(); @@ -155,40 +161,21 @@ public CatalogTable getTable(TablePath tablePath) Connection conn = getConnection(dbUrl); try { DatabaseMetaData metaData = conn.getMetaData(); + Optional primaryKey = getPrimaryKey(metaData, tablePath.getDatabaseName(), tablePath.getTableName()); List constraintKeys = getConstraintKeys( metaData, tablePath.getDatabaseName(), tablePath.getTableName()); - Map columnsDefaultValue = getColumnsDefaultValue(tablePath, conn); - - try (PreparedStatement ps = - conn.prepareStatement( - String.format( - "SELECT * FROM %s WHERE 1 = 0;", - tablePath.getFullNameWithQuoted()))) { - ResultSetMetaData tableMetaData = ps.getMetaData(); - TableSchema.Builder builder = TableSchema.builder(); + String sql = + String.format( + SELECT_COLUMNS, tablePath.getDatabaseName(), tablePath.getTableName()); + try (PreparedStatement ps = conn.prepareStatement(sql); + ResultSet resultSet = ps.executeQuery(); ) { - // add column - for (int i = 1; i <= tableMetaData.getColumnCount(); i++) { - String columnName = tableMetaData.getColumnName(i); - SeaTunnelDataType type = fromJdbcType(tableMetaData, i); - int columnDisplaySize = tableMetaData.getColumnDisplaySize(i); - String comment = tableMetaData.getColumnLabel(i); - boolean isNullable = - tableMetaData.isNullable(i) == ResultSetMetaData.columnNullable; - Object defaultValue = columnsDefaultValue.get(columnName); - - PhysicalColumn physicalColumn = - PhysicalColumn.of( - columnName, - type, - columnDisplaySize, - isNullable, - defaultValue, - comment); - builder.column(physicalColumn); + TableSchema.Builder builder = TableSchema.builder(); + while (resultSet.next()) { + buildTable(resultSet, builder); } // add primary key primaryKey.ifPresent(builder::primaryKey); @@ -202,7 +189,8 @@ public CatalogTable getTable(TablePath tablePath) builder.build(), buildConnectorOptions(tablePath), Collections.emptyList(), - ""); + "", + "mysql"); } } catch (Exception e) { @@ -211,6 +199,67 @@ public CatalogTable getTable(TablePath tablePath) } } + private void buildTable(ResultSet resultSet, TableSchema.Builder builder) throws SQLException { + String columnName = resultSet.getString("COLUMN_NAME"); + String sourceType = resultSet.getString("COLUMN_TYPE"); + String typeName = resultSet.getString("DATA_TYPE").toUpperCase(); + int precision = resultSet.getInt("NUMERIC_PRECISION"); + int scale = resultSet.getInt("NUMERIC_SCALE"); + long columnLength = resultSet.getLong("CHARACTER_MAXIMUM_LENGTH"); + long octetLength = resultSet.getLong("CHARACTER_OCTET_LENGTH"); + if (sourceType.toLowerCase(Locale.ROOT).contains("unsigned")) { + typeName += "_UNSIGNED"; + } + SeaTunnelDataType type = fromJdbcType(typeName, precision, scale); + String comment = resultSet.getString("COLUMN_COMMENT"); + Object defaultValue = resultSet.getObject("COLUMN_DEFAULT"); + String isNullableStr = resultSet.getString("IS_NULLABLE"); + boolean isNullable = isNullableStr.equals("YES"); + long bitLen = 0; + MysqlType mysqlType = MysqlType.valueOf(typeName); + switch (mysqlType) { + case BIT: + bitLen = precision; + break; + case CHAR: + case VARCHAR: + columnLength = octetLength; + break; + case BINARY: + case VARBINARY: + // Uniform conversion to bits + bitLen = octetLength * 4 * 8L; + break; + case BLOB: + case TINYBLOB: + case MEDIUMBLOB: + case LONGBLOB: + bitLen = columnLength << 3; + break; + case JSON: + columnLength = 4 * 1024 * 1024 * 1024L; + break; + default: + break; + } + + PhysicalColumn physicalColumn = + PhysicalColumn.of( + columnName, + type, + 0, + isNullable, + defaultValue, + comment, + sourceType, + sourceType.contains("unsigned"), + sourceType.contains("zerofill"), + bitLen, + null, + columnLength); + builder.column(physicalColumn); + } + public static Map getColumnsDefaultValue(TablePath tablePath, Connection conn) { StringBuilder queryBuf = new StringBuilder("SHOW FULL COLUMNS FROM "); queryBuf.append(StringUtils.quoteIdentifier(tablePath.getTableName(), "`", false)); @@ -235,13 +284,16 @@ public static Map getColumnsDefaultValue(TablePath tablePath, Co } // todo: If the origin source is mysql, we can directly use create table like to create the - // target table? @Override protected boolean createTableInternal(TablePath tablePath, CatalogTable table) throws CatalogException { String dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); - String createTableSql = MysqlCreateTableSqlBuilder.builder(tablePath, table).build(); - try (PreparedStatement ps = getConnection(dbUrl).prepareStatement(createTableSql)) { + + String createTableSql = + MysqlCreateTableSqlBuilder.builder(tablePath, table).build(table.getCatalogName()); + Connection connection = getConnection(dbUrl); + log.info("create table sql: {}", createTableSql); + try (PreparedStatement ps = connection.prepareStatement(createTableSql)) { return ps.execute(); } catch (Exception e) { throw new CatalogException( @@ -252,11 +304,10 @@ protected boolean createTableInternal(TablePath tablePath, CatalogTable table) @Override protected boolean dropTableInternal(TablePath tablePath) throws CatalogException { String dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); + Connection connection = getConnection(dbUrl); try (PreparedStatement ps = - getConnection(dbUrl) - .prepareStatement( - String.format( - "DROP TABLE %s IF EXIST;", tablePath.getFullName()))) { + connection.prepareStatement( + String.format("DROP TABLE IF EXISTS %s;", tablePath.getFullName()))) { // Will there exist concurrent drop for one table? return ps.execute(); } catch (SQLException e) { @@ -309,6 +360,14 @@ private SeaTunnelDataType fromJdbcType(ResultSetMetaData metadata, int colInd return new MysqlDataTypeConvertor().toSeaTunnelType(mysqlType, dataTypeProperties); } + private SeaTunnelDataType fromJdbcType(String typeName, int precision, int scale) { + MysqlType mysqlType = MysqlType.getByName(typeName); + Map dataTypeProperties = new HashMap<>(); + dataTypeProperties.put(MysqlDataTypeConvertor.PRECISION, precision); + dataTypeProperties.put(MysqlDataTypeConvertor.SCALE, scale); + return new MysqlDataTypeConvertor().toSeaTunnelType(mysqlType, dataTypeProperties); + } + @SuppressWarnings("MagicNumber") private Map buildConnectorOptions(TablePath tablePath) { Map options = new HashMap<>(8); @@ -321,6 +380,7 @@ private Map buildConnectorOptions(TablePath tablePath) { } private String getUrlFromDatabaseName(String databaseName) { - return baseUrl + databaseName + suffix; + String url = baseUrl.endsWith("/") ? baseUrl : baseUrl + "/"; + return url + databaseName + suffix; } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogFactory.java index 014af151c8e7..8d3a76bed693 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogFactory.java @@ -37,9 +37,11 @@ @AutoService(Factory.class) public class MySqlCatalogFactory implements CatalogFactory { + public static final String IDENTIFIER = "MySQL"; + @Override public String factoryIdentifier() { - return "MySQL"; + return IDENTIFIER; } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilder.java index 9a015ca73951..cec934bcb016 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlCreateTableSqlBuilder.java @@ -23,10 +23,14 @@ import org.apache.seatunnel.api.table.catalog.PrimaryKey; import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.SqlType; import org.apache.commons.collections4.CollectionUtils; import org.apache.commons.lang3.StringUtils; +import com.mysql.cj.MysqlType; + import java.util.ArrayList; import java.util.List; import java.util.stream.Collectors; @@ -111,12 +115,12 @@ public MysqlCreateTableSqlBuilder comment(String comment) { return this; } - public String build() { + public String build(String catalogName) { List sqls = new ArrayList<>(); sqls.add( String.format( "CREATE TABLE IF NOT EXISTS %s (\n%s\n)", - tableName, buildColumnsIdentifySql())); + tableName, buildColumnsIdentifySql(catalogName))); if (engine != null) { sqls.add("ENGINE = " + engine); } @@ -132,10 +136,10 @@ public String build() { return String.join(" ", sqls) + ";"; } - private String buildColumnsIdentifySql() { + private String buildColumnsIdentifySql(String catalogName) { List columnSqls = new ArrayList<>(); for (Column column : columns) { - columnSqls.add("\t" + buildColumnIdentifySql(column)); + columnSqls.add("\t" + buildColumnIdentifySql(column, catalogName)); } if (primaryKey != null) { columnSqls.add("\t" + buildPrimaryKeySql()); @@ -145,22 +149,79 @@ private String buildColumnsIdentifySql() { if (StringUtils.isBlank(constraintKey.getConstraintName())) { continue; } - columnSqls.add("\t" + buildConstraintKeySql(constraintKey)); + // columnSqls.add("\t" + buildConstraintKeySql(constraintKey)); } } return String.join(", \n", columnSqls); } - private String buildColumnIdentifySql(Column column) { + private String buildColumnIdentifySql(Column column, String catalogName) { final List columnSqls = new ArrayList<>(); - // Column name columnSqls.add(column.getName()); - // Column type - columnSqls.add( - mysqlDataTypeConvertor.toConnectorType(column.getDataType(), null).getName()); - // Column length - if (column.getColumnLength() != null) { - columnSqls.add("(" + column.getColumnLength() + ")"); + if (StringUtils.equals(catalogName, "mysql")) { + columnSqls.add(column.getSourceType()); + } else { + // Column name + SqlType dataType = column.getDataType().getSqlType(); + boolean isBytes = StringUtils.equals(dataType.name(), SqlType.BYTES.name()); + Long columnLength = column.getLongColumnLength(); + Long bitLen = column.getBitLen(); + if (isBytes) { + if (bitLen >= 0 && bitLen <= 64) { + columnSqls.add(MysqlType.BIT.getName()); + columnSqls.add("(" + (bitLen == 0 ? 1 : bitLen) + ")"); + } else { + bitLen = bitLen == -1 ? bitLen : bitLen >> 3; + if (bitLen >= 0 && bitLen <= 255) { + columnSqls.add(MysqlType.TINYBLOB.getName()); + } else if (bitLen <= 16383) { + columnSqls.add(MysqlType.BLOB.getName()); + } else if (bitLen <= 16777215) { + columnSqls.add(MysqlType.MEDIUMBLOB.getName()); + } else { + columnSqls.add(MysqlType.LONGBLOB.getName()); + } + } + } else { + if (columnLength >= 16383 && columnLength <= 65535) { + columnSqls.add(MysqlType.TEXT.getName()); + } else if (columnLength >= 65535 && columnLength <= 16777215) { + columnSqls.add(MysqlType.MEDIUMTEXT.getName()); + } else if (columnLength > 16777215 || columnLength == -1) { + columnSqls.add(MysqlType.LONGTEXT.getName()); + } else { + // Column type + columnSqls.add( + mysqlDataTypeConvertor + .toConnectorType(column.getDataType(), null) + .getName()); + // Column length + // add judge is need column legth + if (column.getColumnLength() != null) { + final String name = + mysqlDataTypeConvertor + .toConnectorType(column.getDataType(), null) + .getName(); + String fieSql = ""; + List list = new ArrayList<>(); + list.add(MysqlType.VARCHAR.getName()); + list.add(MysqlType.CHAR.getName()); + list.add(MysqlType.BIGINT.getName()); + list.add(MysqlType.INT.getName()); + if (StringUtils.equals(name, MysqlType.DECIMAL.getName())) { + DecimalType decimalType = (DecimalType) column.getDataType(); + fieSql = + String.format( + "(%d, %d)", + decimalType.getPrecision(), decimalType.getScale()); + columnSqls.add(fieSql); + } else if (list.contains(name)) { + fieSql = "(" + column.getLongColumnLength() + ")"; + columnSqls.add(fieSql); + } + } + } + } } // nullable if (column.isNullable()) { @@ -168,14 +229,11 @@ private String buildColumnIdentifySql(Column column) { } else { columnSqls.add("NOT NULL"); } - // default value - if (column.getDefaultValue() != null) { - columnSqls.add("DEFAULT '" + column.getDefaultValue() + "'"); - } - // comment + // TODO support default value if (column.getComment() != null) { columnSqls.add("COMMENT '" + column.getComment() + "'"); } + return String.join(" ", columnSqls); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlDataTypeConvertor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlDataTypeConvertor.java index e30025c4e712..16e5b87d3037 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlDataTypeConvertor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MysqlDataTypeConvertor.java @@ -168,7 +168,9 @@ public MysqlType toConnectorType( SqlType sqlType = seaTunnelDataType.getSqlType(); // todo: verify switch (sqlType) { - case ARRAY: + // from pg array not support + // case ARRAY: + // return MysqlType.ENUM; case MAP: case ROW: case STRING: @@ -196,9 +198,9 @@ public MysqlType toConnectorType( case DATE: return MysqlType.DATE; case TIME: - return MysqlType.DATETIME; + return MysqlType.TIME; case TIMESTAMP: - return MysqlType.TIMESTAMP; + return MysqlType.DATETIME; default: throw new JdbcConnectorException( CommonErrorCode.UNSUPPORTED_DATA_TYPE, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java new file mode 100644 index 000000000000..261f4f7fb6ff --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalog.java @@ -0,0 +1,361 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; + +import lombok.extern.slf4j.Slf4j; + +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_BFILE; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_BLOB; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_CHAR; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_CLOB; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_LONG; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_LONG_RAW; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_NCHAR; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_NCLOB; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_NVARCHAR2; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_RAW; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_ROWID; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleDataTypeConvertor.ORACLE_VARCHAR2; + +@Slf4j +public class OracleCatalog extends AbstractJdbcCatalog { + private static final OracleDataTypeConvertor DATA_TYPE_CONVERTOR = + new OracleDataTypeConvertor(); + private static final List EXCLUDED_SCHEMAS = + Collections.unmodifiableList( + Arrays.asList( + "APPQOSSYS", + "AUDSYS", + "CTXSYS", + "DVSYS", + "DBSFWUSER", + "DBSNMP", + "GSMADMIN_INTERNAL", + "LBACSYS", + "MDSYS", + "OJVMSYS", + "OLAPSYS", + "ORDDATA", + "ORDSYS", + "OUTLN", + "SYS", + "SYSTEM", + "WMSYS", + "XDB", + "EXFSYS", + "SYSMAN")); + + private static final String SELECT_COLUMNS_SQL = + "SELECT\n" + + " cols.COLUMN_NAME,\n" + + " CASE \n" + + " WHEN cols.data_type LIKE 'INTERVAL%%' THEN 'INTERVAL'\n" + + " ELSE REGEXP_SUBSTR(cols.data_type, '^[^(]+')\n" + + " END as TYPE_NAME,\n" + + " cols.data_type || \n" + + " CASE \n" + + " WHEN cols.data_type IN ('VARCHAR2', 'CHAR') THEN '(' || cols.data_length || ')'\n" + + " WHEN cols.data_type IN ('NUMBER') AND cols.data_precision IS NOT NULL AND cols.data_scale IS NOT NULL THEN '(' || cols.data_precision || ', ' || cols.data_scale || ')'\n" + + " WHEN cols.data_type IN ('NUMBER') AND cols.data_precision IS NOT NULL AND cols.data_scale IS NULL THEN '(' || cols.data_precision || ')'\n" + + " WHEN cols.data_type IN ('RAW') THEN '(' || cols.data_length || ')'\n" + + " END AS FULL_TYPE_NAME,\n" + + " cols.data_length AS COLUMN_LENGTH,\n" + + " cols.data_precision AS COLUMN_PRECISION,\n" + + " cols.data_scale AS COLUMN_SCALE,\n" + + " com.comments AS COLUMN_COMMENT,\n" + + " cols.data_default AS DEFAULT_VALUE,\n" + + " CASE cols.nullable WHEN 'N' THEN 'NO' ELSE 'YES' END AS IS_NULLABLE\n" + + "FROM\n" + + " all_tab_columns cols\n" + + "LEFT JOIN \n" + + " all_col_comments com ON cols.table_name = com.table_name AND cols.column_name = com.column_name AND cols.owner = com.owner\n" + + "WHERE \n" + + " cols.owner = '%s'\n" + + " AND cols.table_name = '%s'\n" + + "ORDER BY \n" + + " cols.column_id \n"; + + public OracleCatalog( + String catalogName, + String username, + String pwd, + JdbcUrlUtil.UrlInfo urlInfo, + String defaultSchema) { + super(catalogName, username, pwd, urlInfo, defaultSchema); + } + + @Override + public List listDatabases() throws CatalogException { + try (PreparedStatement ps = + defaultConnection.prepareStatement("SELECT name FROM v$database")) { + + List databases = new ArrayList<>(); + ResultSet rs = ps.executeQuery(); + + while (rs.next()) { + String databaseName = rs.getString(1); + databases.add(databaseName); + } + return databases; + } catch (Exception e) { + throw new CatalogException( + String.format("Failed listing database in catalog %s", this.catalogName), e); + } + } + + @Override + protected boolean createTableInternal(TablePath tablePath, CatalogTable table) + throws CatalogException { + String createTableSql = new OracleCreateTableSqlBuilder(table).build(tablePath); + String[] createTableSqls = createTableSql.split(";"); + for (String sql : createTableSqls) { + log.info("create table sql: {}", sql); + try (PreparedStatement ps = defaultConnection.prepareStatement(sql)) { + ps.execute(); + } catch (Exception e) { + throw new CatalogException( + String.format("Failed creating table %s", tablePath.getFullName()), e); + } + } + return true; + } + + @Override + protected boolean dropTableInternal(TablePath tablePath) throws CatalogException { + return false; + } + + @Override + protected boolean createDatabaseInternal(String databaseName) { + return false; + } + + @Override + protected boolean dropDatabaseInternal(String databaseName) throws CatalogException { + return false; + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + try { + return databaseExists(tablePath.getDatabaseName()) + && listTables(tablePath.getDatabaseName()) + .contains(tablePath.getSchemaAndTableName().toUpperCase()); + } catch (DatabaseNotExistException e) { + return false; + } + } + + @Override + public List listTables(String databaseName) + throws CatalogException, DatabaseNotExistException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(this.catalogName, databaseName); + } + + try (PreparedStatement ps = + defaultConnection.prepareStatement( + "SELECT OWNER, TABLE_NAME FROM ALL_TABLES\n" + + "WHERE TABLE_NAME NOT LIKE 'MDRT_%'\n" + + " AND TABLE_NAME NOT LIKE 'MDRS_%'\n" + + " AND TABLE_NAME NOT LIKE 'MDXT_%'\n" + + " AND (TABLE_NAME NOT LIKE 'SYS_IOT_OVER_%' AND IOT_NAME IS NULL)")) { + + ResultSet rs = ps.executeQuery(); + List tables = new ArrayList<>(); + while (rs.next()) { + if (EXCLUDED_SCHEMAS.contains(rs.getString(1))) { + continue; + } + tables.add(rs.getString(1) + "." + rs.getString(2)); + } + + return tables; + } catch (Exception e) { + throw new CatalogException( + String.format("Failed listing database in catalog %s", catalogName), e); + } + } + + @Override + public CatalogTable getTable(TablePath tablePath) + throws CatalogException, TableNotExistException { + if (!tableExists(tablePath)) { + throw new TableNotExistException(catalogName, tablePath); + } + + try { + DatabaseMetaData metaData = defaultConnection.getMetaData(); + Optional primaryKey = + getPrimaryKey( + metaData, + tablePath.getDatabaseName(), + tablePath.getSchemaName(), + tablePath.getTableName()); + List constraintKeys = + getConstraintKeys( + metaData, + tablePath.getDatabaseName(), + tablePath.getSchemaName(), + tablePath.getTableName()); + + String sql = + String.format( + SELECT_COLUMNS_SQL, + tablePath.getSchemaName(), + tablePath.getTableName()); + try (PreparedStatement ps = defaultConnection.prepareStatement(sql); + ResultSet resultSet = ps.executeQuery()) { + TableSchema.Builder builder = TableSchema.builder(); + // add column + while (resultSet.next()) { + buildColumn(resultSet, builder); + } + + // add primary key + primaryKey.ifPresent(builder::primaryKey); + // add constraint key + constraintKeys.forEach(builder::constraintKey); + TableIdentifier tableIdentifier = + TableIdentifier.of( + catalogName, + tablePath.getDatabaseName(), + tablePath.getSchemaName(), + tablePath.getTableName()); + return CatalogTable.of( + tableIdentifier, + builder.build(), + buildConnectorOptions(tablePath), + Collections.emptyList(), + ""); + } + + } catch (Exception e) { + throw new CatalogException( + String.format("Failed getting table %s", tablePath.getFullName()), e); + } + } + + private void buildColumn(ResultSet resultSet, TableSchema.Builder builder) throws SQLException { + String columnName = resultSet.getString("COLUMN_NAME"); + String typeName = resultSet.getString("TYPE_NAME"); + String fullTypeName = resultSet.getString("FULL_TYPE_NAME"); + long columnLength = resultSet.getLong("COLUMN_LENGTH"); + long columnPrecision = resultSet.getLong("COLUMN_PRECISION"); + long columnScale = resultSet.getLong("COLUMN_SCALE"); + String columnComment = resultSet.getString("COLUMN_COMMENT"); + Object defaultValue = resultSet.getObject("DEFAULT_VALUE"); + boolean isNullable = resultSet.getString("IS_NULLABLE").equals("YES"); + + SeaTunnelDataType type = fromJdbcType(typeName, columnPrecision, columnScale); + long bitLen = 0; + switch (typeName) { + case ORACLE_LONG: + case ORACLE_ROWID: + case ORACLE_NCLOB: + case ORACLE_CLOB: + columnLength = -1; + break; + case ORACLE_RAW: + bitLen = 2000 * 8; + break; + case ORACLE_BLOB: + case ORACLE_LONG_RAW: + case ORACLE_BFILE: + bitLen = -1; + break; + case ORACLE_CHAR: + case ORACLE_NCHAR: + case ORACLE_NVARCHAR2: + case ORACLE_VARCHAR2: + default: + break; + } + + PhysicalColumn physicalColumn = + PhysicalColumn.of( + columnName, + type, + 0, + isNullable, + defaultValue, + columnComment, + fullTypeName, + false, + false, + bitLen, + null, + columnLength); + builder.column(physicalColumn); + } + + @SuppressWarnings("unchecked") + private SeaTunnelDataType fromJdbcType(ResultSetMetaData metadata, int colIndex) + throws SQLException { + String columnType = metadata.getColumnTypeName(colIndex); + Map dataTypeProperties = new HashMap<>(); + dataTypeProperties.put(OracleDataTypeConvertor.PRECISION, metadata.getPrecision(colIndex)); + dataTypeProperties.put(OracleDataTypeConvertor.SCALE, metadata.getScale(colIndex)); + return DATA_TYPE_CONVERTOR.toSeaTunnelType(columnType, dataTypeProperties); + } + + private SeaTunnelDataType fromJdbcType(String typeName, long precision, long scale) { + Map dataTypeProperties = new HashMap<>(); + dataTypeProperties.put(OracleDataTypeConvertor.PRECISION, precision); + dataTypeProperties.put(OracleDataTypeConvertor.SCALE, scale); + return DATA_TYPE_CONVERTOR.toSeaTunnelType(typeName, dataTypeProperties); + } + + @SuppressWarnings("MagicNumber") + private Map buildConnectorOptions(TablePath tablePath) { + Map options = new HashMap<>(8); + options.put("connector", "jdbc"); + options.put("url", baseUrl); + options.put("table-name", tablePath.getSchemaAndTableName()); + options.put("username", username); + options.put("password", pwd); + return options; + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java new file mode 100644 index 000000000000..4ea5242835cd --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogFactory.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.configuration.util.OptionValidationException; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; + +import com.google.auto.service.AutoService; + +import java.util.Optional; + +@AutoService(Factory.class) +public class OracleCatalogFactory implements CatalogFactory { + + @Override + public String factoryIdentifier() { + return "Oracle"; + } + + @Override + public Catalog createCatalog(String catalogName, ReadonlyConfig options) { + String urlWithDatabase = options.get(JdbcCatalogOptions.BASE_URL); + JdbcUrlUtil.UrlInfo urlInfo = OracleURLParser.parse(urlWithDatabase); + Optional defaultDatabase = urlInfo.getDefaultDatabase(); + if (!defaultDatabase.isPresent()) { + throw new OptionValidationException(JdbcCatalogOptions.BASE_URL); + } + return new OracleCatalog( + catalogName, + options.get(JdbcCatalogOptions.USERNAME), + options.get(JdbcCatalogOptions.PASSWORD), + urlInfo, + options.get(JdbcCatalogOptions.SCHEMA)); + } + + @Override + public OptionRule optionRule() { + return JdbcCatalogOptions.BASE_RULE.build(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilder.java new file mode 100644 index 000000000000..984dd93e6a67 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCreateTableSqlBuilder.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.SqlType; + +import org.apache.commons.lang3.StringUtils; + +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; + +public class OracleCreateTableSqlBuilder { + + private List columns; + private PrimaryKey primaryKey; + private OracleDataTypeConvertor oracleDataTypeConvertor; + private String sourceCatalogName; + + public OracleCreateTableSqlBuilder(CatalogTable catalogTable) { + this.columns = catalogTable.getTableSchema().getColumns(); + this.primaryKey = catalogTable.getTableSchema().getPrimaryKey(); + this.oracleDataTypeConvertor = new OracleDataTypeConvertor(); + this.sourceCatalogName = catalogTable.getCatalogName(); + } + + public String build(TablePath tablePath) { + StringBuilder createTableSql = new StringBuilder(); + createTableSql + .append("CREATE TABLE ") + .append(tablePath.getSchemaAndTableName()) + .append(" (\n"); + + List columnSqls = + columns.stream().map(this::buildColumnSql).collect(Collectors.toList()); + + // Add primary key directly in the create table statement + if (primaryKey != null + && primaryKey.getColumnNames() != null + && primaryKey.getColumnNames().size() > 0) { + columnSqls.add(buildPrimaryKeySql(primaryKey)); + } + + createTableSql.append(String.join(",\n", columnSqls)); + createTableSql.append("\n)"); + + List commentSqls = + columns.stream() + .filter(column -> StringUtils.isNotBlank(column.getComment())) + .map( + column -> + buildColumnCommentSql( + column, tablePath.getSchemaAndTableName())) + .collect(Collectors.toList()); + + if (!commentSqls.isEmpty()) { + createTableSql.append(";\n"); + createTableSql.append(String.join(";\n", commentSqls)); + } + + return createTableSql.toString(); + } + + private String buildColumnSql(Column column) { + StringBuilder columnSql = new StringBuilder(); + columnSql.append(column.getName()).append(" "); + + String columnType = + sourceCatalogName.equals("oracle") + ? column.getSourceType() + : buildColumnType(column); + columnSql.append(columnType); + + if (!column.isNullable()) { + columnSql.append(" NOT NULL"); + } + + // if (column.getDefaultValue() != null) { + // columnSql.append(" DEFAULT + // '").append(column.getDefaultValue().toString()).append("'"); + // } + + return columnSql.toString(); + } + + private String buildColumnType(Column column) { + SqlType sqlType = column.getDataType().getSqlType(); + Long columnLength = column.getLongColumnLength(); + Long bitLen = column.getBitLen(); + switch (sqlType) { + case BYTES: + if (bitLen < 0 || bitLen > 2000) { + return "BLOB"; + } else { + return "RAW(" + bitLen + ")"; + } + case STRING: + if (columnLength > 0 && columnLength < 4000) { + return "VARCHAR2(" + columnLength + " CHAR)"; + } else { + return "CLOB"; + } + default: + String type = oracleDataTypeConvertor.toConnectorType(column.getDataType(), null); + if (type.equals("NUMBER")) { + if (column.getDataType() instanceof DecimalType) { + DecimalType decimalType = (DecimalType) column.getDataType(); + return "NUMBER(" + + decimalType.getPrecision() + + "," + + decimalType.getScale() + + ")"; + } else { + return "NUMBER"; + } + } + return type; + } + } + + private String buildPrimaryKeySql(PrimaryKey primaryKey) { + String randomSuffix = UUID.randomUUID().toString().replace("-", "").substring(0, 4); + String columnNamesString = String.join(", ", primaryKey.getColumnNames()); + + // In Oracle database, the maximum length for an identifier is 30 characters. + String primaryKeyStr = primaryKey.getPrimaryKey(); + if (primaryKeyStr.length() > 25) { + primaryKeyStr = primaryKeyStr.substring(0, 25); + } + + return "CONSTRAINT " + + primaryKeyStr + + "_" + + randomSuffix + + " PRIMARY KEY (" + + columnNamesString + + ")"; + } + + private String buildColumnCommentSql(Column column, String tableName) { + StringBuilder columnCommentSql = new StringBuilder(); + columnCommentSql.append("COMMENT ON COLUMN ").append(tableName).append("."); + columnCommentSql + .append(column.getName()) + .append(" IS '") + .append(column.getComment()) + .append("'"); + return columnCommentSql.toString(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleDataTypeConvertor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleDataTypeConvertor.java new file mode 100644 index 000000000000..cd42a05e0d33 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleDataTypeConvertor.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle; + +import org.apache.seatunnel.api.table.catalog.DataTypeConvertException; +import org.apache.seatunnel.api.table.catalog.DataTypeConvertor; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; + +import org.apache.commons.collections4.MapUtils; + +import com.google.auto.service.AutoService; + +import java.util.Collections; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static com.google.common.base.Preconditions.checkNotNull; + +@AutoService(DataTypeConvertor.class) +public class OracleDataTypeConvertor implements DataTypeConvertor { + + public static final String PRECISION = "precision"; + public static final String SCALE = "scale"; + public static final Integer DEFAULT_PRECISION = 38; + public static final Integer DEFAULT_SCALE = 18; + + // ============================data types===================== + public static final String ORACLE_UNKNOWN = "UNKNOWN"; + // -------------------------number---------------------------- + public static final String ORACLE_BINARY_DOUBLE = "BINARY_DOUBLE"; + public static final String ORACLE_BINARY_FLOAT = "BINARY_FLOAT"; + public static final String ORACLE_NUMBER = "NUMBER"; + public static final String ORACLE_FLOAT = "FLOAT"; + public static final String ORACLE_REAL = "REAL"; + public static final String ORACLE_INTEGER = "INTEGER"; + // -------------------------string---------------------------- + public static final String ORACLE_CHAR = "CHAR"; + public static final String ORACLE_VARCHAR2 = "VARCHAR2"; + public static final String ORACLE_NCHAR = "NCHAR"; + public static final String ORACLE_NVARCHAR2 = "NVARCHAR2"; + public static final String ORACLE_LONG = "LONG"; + public static final String ORACLE_ROWID = "ROWID"; + public static final String ORACLE_CLOB = "CLOB"; + public static final String ORACLE_NCLOB = "NCLOB"; + // ------------------------------time------------------------- + public static final String ORACLE_DATE = "DATE"; + public static final String ORACLE_TIMESTAMP = "TIMESTAMP"; + public static final String ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE = + "TIMESTAMP WITH LOCAL TIME ZONE"; + // ------------------------------blob------------------------- + public static final String ORACLE_BLOB = "BLOB"; + public static final String ORACLE_BFILE = "BFILE"; + public static final String ORACLE_RAW = "RAW"; + public static final String ORACLE_LONG_RAW = "LONG RAW"; + + @Override + public SeaTunnelDataType toSeaTunnelType(String connectorDataType) { + return toSeaTunnelType(connectorDataType, Collections.emptyMap()); + } + + @Override + public SeaTunnelDataType toSeaTunnelType( + String connectorDataType, Map dataTypeProperties) + throws DataTypeConvertException { + checkNotNull(connectorDataType, "Oracle Type cannot be null"); + connectorDataType = normalizeTimestamp(connectorDataType); + switch (connectorDataType) { + case ORACLE_INTEGER: + return BasicType.INT_TYPE; + case ORACLE_FLOAT: + // The float type will be converted to DecimalType(10, -127), + // which will lose precision in the spark engine + return new DecimalType(38, 18); + case ORACLE_NUMBER: + int precision = + MapUtils.getInteger(dataTypeProperties, PRECISION, DEFAULT_PRECISION); + int scale = MapUtils.getInteger(dataTypeProperties, SCALE, DEFAULT_SCALE); + if (scale == 0) { + if (precision == 1) { + return BasicType.BOOLEAN_TYPE; + } + if (precision <= 9) { + return BasicType.INT_TYPE; + } + if (precision <= 18) { + return BasicType.LONG_TYPE; + } + } + return new DecimalType(38, 18); + case ORACLE_BINARY_DOUBLE: + return BasicType.DOUBLE_TYPE; + case ORACLE_BINARY_FLOAT: + case ORACLE_REAL: + return BasicType.FLOAT_TYPE; + case ORACLE_CHAR: + case ORACLE_NCHAR: + case ORACLE_NVARCHAR2: + case ORACLE_VARCHAR2: + case ORACLE_LONG: + case ORACLE_ROWID: + case ORACLE_NCLOB: + case ORACLE_CLOB: + return BasicType.STRING_TYPE; + case ORACLE_DATE: + return LocalTimeType.LOCAL_DATE_TYPE; + case ORACLE_TIMESTAMP: + case ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE: + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + case ORACLE_BLOB: + case ORACLE_RAW: + case ORACLE_LONG_RAW: + case ORACLE_BFILE: + return PrimitiveByteArrayType.INSTANCE; + // Doesn't support yet + case ORACLE_UNKNOWN: + default: + throw new JdbcConnectorException( + CommonErrorCode.UNSUPPORTED_OPERATION, + String.format("Doesn't support ORACLE type '%s' yet.", connectorDataType)); + } + } + + @Override + public String toConnectorType( + SeaTunnelDataType seaTunnelDataType, Map dataTypeProperties) + throws DataTypeConvertException { + checkNotNull(seaTunnelDataType, "seaTunnelDataType cannot be null"); + SqlType sqlType = seaTunnelDataType.getSqlType(); + switch (sqlType) { + case TINYINT: + case SMALLINT: + case INT: + return ORACLE_INTEGER; + case BIGINT: + return ORACLE_NUMBER; + case FLOAT: + return ORACLE_FLOAT; + case DOUBLE: + return ORACLE_BINARY_DOUBLE; + case DECIMAL: + return ORACLE_NUMBER; + case BOOLEAN: + return ORACLE_NUMBER; + case STRING: + return ORACLE_VARCHAR2; + case DATE: + return ORACLE_DATE; + case TIMESTAMP: + return ORACLE_TIMESTAMP_WITH_LOCAL_TIME_ZONE; + case BYTES: + return ORACLE_BLOB; + default: + throw new UnsupportedOperationException( + String.format( + "Doesn't support SeaTunnel type '%s' yet.", seaTunnelDataType)); + } + } + + public static String normalizeTimestamp(String oracleType) { + // Create a pattern to match TIMESTAMP followed by an optional (0-9) + String pattern = "^TIMESTAMP(\\([0-9]\\))?$"; + // Create a Pattern object + Pattern r = Pattern.compile(pattern); + // Now create matcher object. + Matcher m = r.matcher(oracleType); + if (m.find()) { + return "TIMESTAMP"; + } else { + return oracleType; + } + } + + @Override + public String getIdentity() { + return "Oracle"; + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleURLParser.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleURLParser.java new file mode 100644 index 000000000000..adcb5236de5a --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleURLParser.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle; + +import org.apache.seatunnel.common.utils.JdbcUrlUtil; + +import java.util.Optional; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class OracleURLParser { + private static final Pattern ORACLE_URL_PATTERN = + Pattern.compile( + "^(?jdbc:oracle:thin:@(//)?(?[^:]+):(?\\d+)[:/])(?.+?)((?\\?.*)?)$"); + + public static JdbcUrlUtil.UrlInfo parse(String url) { + Matcher matcher = ORACLE_URL_PATTERN.matcher(url); + if (matcher.find()) { + String urlWithoutDatabase = matcher.group("url"); + String host = matcher.group("host"); + Integer port = Integer.valueOf(matcher.group("port")); + String database = matcher.group("database"); + String suffix = Optional.ofNullable(matcher.group("suffix")).orElse(""); + return new JdbcUrlUtil.UrlInfo(url, urlWithoutDatabase, host, port, database, suffix); + } + throw new IllegalArgumentException("The jdbc url format is incorrect: " + url); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java new file mode 100644 index 000000000000..e3507666d08f --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalog.java @@ -0,0 +1,441 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PhysicalColumn; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.catalog.exception.CatalogException; +import org.apache.seatunnel.api.table.catalog.exception.DatabaseNotExistException; +import org.apache.seatunnel.api.table.catalog.exception.TableNotExistException; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; + +import com.mysql.cj.MysqlType; +import com.mysql.cj.jdbc.result.ResultSetImpl; +import lombok.extern.slf4j.Slf4j; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_BIT; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_BYTEA; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_CHAR; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_CHARACTER; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_CHARACTER_VARYING; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_GEOGRAPHY; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_GEOMETRY; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_INTERVAL; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_TEXT; + +@Slf4j +public class PostgresCatalog extends AbstractJdbcCatalog { + + private static final String SELECT_COLUMNS_SQL = + "SELECT \n" + + " a.attname AS column_name, \n" + + "\t\tt.typname as type_name,\n" + + " CASE \n" + + " WHEN t.typname = 'varchar' THEN t.typname || '(' || (a.atttypmod - 4) || ')'\n" + + " WHEN t.typname = 'bpchar' THEN 'char' || '(' || (a.atttypmod - 4) || ')'\n" + + " WHEN t.typname = 'numeric' OR t.typname = 'decimal' THEN t.typname || '(' || ((a.atttypmod - 4) >> 16) || ', ' || ((a.atttypmod - 4) & 65535) || ')'\n" + + " WHEN t.typname = 'bit' OR t.typname = 'bit varying' THEN t.typname || '(' || (a.atttypmod - 4) || ')'\n" + + " ELSE t.typname\n" + + " END AS full_type_name,\n" + + " CASE\n" + + " WHEN t.typname IN ('varchar', 'bpchar', 'bit', 'bit varying') THEN a.atttypmod - 4\n" + + " WHEN t.typname IN ('numeric', 'decimal') THEN (a.atttypmod - 4) >> 16\n" + + " ELSE NULL\n" + + " END AS column_length,\n" + + "\t\tCASE\n" + + " WHEN t.typname IN ('numeric', 'decimal') THEN (a.atttypmod - 4) & 65535\n" + + " ELSE NULL\n" + + " END AS column_scale,\n" + + "\t\td.description AS column_comment,\n" + + "\t\tpg_get_expr(ad.adbin, ad.adrelid) AS default_value,\n" + + "\t\tCASE WHEN a.attnotnull THEN 'NO' ELSE 'YES' END AS is_nullable\n" + + "FROM \n" + + " pg_class c\n" + + " JOIN pg_namespace n ON c.relnamespace = n.oid\n" + + " JOIN pg_attribute a ON a.attrelid = c.oid\n" + + " JOIN pg_type t ON a.atttypid = t.oid\n" + + " LEFT JOIN pg_description d ON c.oid = d.objoid AND a.attnum = d.objsubid\n" + + " LEFT JOIN pg_attrdef ad ON a.attnum = ad.adnum AND a.attrelid = ad.adrelid\n" + + "WHERE \n" + + " n.nspname = '%s'\n" + + " AND c.relname = '%s'\n" + + " AND a.attnum > 0\n" + + "ORDER BY \n" + + " a.attnum;"; + + protected static final Set SYS_DATABASES = new HashSet<>(9); + + static { + SYS_DATABASES.add("information_schema"); + SYS_DATABASES.add("pg_catalog"); + SYS_DATABASES.add("root"); + SYS_DATABASES.add("pg_toast"); + SYS_DATABASES.add("pg_temp_1"); + SYS_DATABASES.add("pg_toast_temp_1"); + SYS_DATABASES.add("postgres"); + SYS_DATABASES.add("template0"); + SYS_DATABASES.add("template1"); + } + + protected final Map connectionMap; + + public PostgresCatalog( + String catalogName, + String username, + String pwd, + JdbcUrlUtil.UrlInfo urlInfo, + String defaultSchema) { + super(catalogName, username, pwd, urlInfo, defaultSchema); + this.connectionMap = new ConcurrentHashMap<>(); + } + + public Connection getConnection(String url) { + if (connectionMap.containsKey(url)) { + return connectionMap.get(url); + } + try { + Connection connection = DriverManager.getConnection(url, username, pwd); + connectionMap.put(url, connection); + return connection; + } catch (SQLException e) { + throw new CatalogException(String.format("Failed connecting to %s via JDBC.", url), e); + } + } + + @Override + public void close() throws CatalogException { + for (Map.Entry entry : connectionMap.entrySet()) { + try { + entry.getValue().close(); + } catch (SQLException e) { + throw new CatalogException( + String.format("Failed to close %s via JDBC.", entry.getKey()), e); + } + } + super.close(); + } + + @Override + public List listDatabases() throws CatalogException { + try (PreparedStatement ps = + defaultConnection.prepareStatement("select datname from pg_database;")) { + + List databases = new ArrayList<>(); + ResultSet rs = ps.executeQuery(); + + while (rs.next()) { + String databaseName = rs.getString(1); + if (!SYS_DATABASES.contains(databaseName)) { + databases.add(rs.getString(1)); + } + } + + return databases; + } catch (Exception e) { + throw new CatalogException( + String.format("Failed listing database in catalog %s", this.catalogName), e); + } + } + + @Override + public List listTables(String databaseName) + throws CatalogException, DatabaseNotExistException { + if (!databaseExists(databaseName)) { + throw new DatabaseNotExistException(this.catalogName, databaseName); + } + + String dbUrl = getUrlFromDatabaseName(databaseName); + Connection connection = getConnection(dbUrl); + try (PreparedStatement ps = + connection.prepareStatement( + "SELECT table_schema, table_name FROM information_schema.tables;")) { + + ResultSet rs = ps.executeQuery(); + + List tables = new ArrayList<>(); + + while (rs.next()) { + String schemaName = rs.getString("table_schema"); + String tableName = rs.getString("table_name"); + if (org.apache.commons.lang3.StringUtils.isNotBlank(schemaName) + && !SYS_DATABASES.contains(schemaName)) { + tables.add(schemaName + "." + tableName); + } + } + + return tables; + } catch (Exception e) { + throw new CatalogException( + String.format("Failed listing database in catalog %s", catalogName), e); + } + } + + @Override + public CatalogTable getTable(TablePath tablePath) + throws CatalogException, TableNotExistException { + if (!tableExists(tablePath)) { + throw new TableNotExistException(catalogName, tablePath); + } + + String dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); + Connection conn = getConnection(dbUrl); + try { + DatabaseMetaData metaData = conn.getMetaData(); + Optional primaryKey = + getPrimaryKey( + metaData, + tablePath.getDatabaseName(), + tablePath.getSchemaName(), + tablePath.getTableName()); + List constraintKeys = + getConstraintKeys( + metaData, + tablePath.getDatabaseName(), + tablePath.getSchemaName(), + tablePath.getTableName()); + + String sql = + String.format( + SELECT_COLUMNS_SQL, + tablePath.getSchemaName(), + tablePath.getTableName()); + try (PreparedStatement ps = conn.prepareStatement(sql); + ResultSet resultSet = ps.executeQuery()) { + TableSchema.Builder builder = TableSchema.builder(); + + // add column + while (resultSet.next()) { + buildColumn(resultSet, builder); + } + + // add primary key + primaryKey.ifPresent(builder::primaryKey); + // add constraint key + constraintKeys.forEach(builder::constraintKey); + TableIdentifier tableIdentifier = + TableIdentifier.of( + catalogName, + tablePath.getDatabaseName(), + tablePath.getSchemaName(), + tablePath.getTableName()); + return CatalogTable.of( + tableIdentifier, + builder.build(), + buildConnectorOptions(tablePath), + Collections.emptyList(), + "", + "postgres"); + } + + } catch (Exception e) { + throw new CatalogException( + String.format("Failed getting table %s", tablePath.getFullName()), e); + } + } + + private void buildColumn(ResultSet resultSet, TableSchema.Builder builder) throws SQLException { + String columnName = resultSet.getString("column_name"); + String typeName = resultSet.getString("type_name"); + String fullTypeName = resultSet.getString("full_type_name"); + long columnLength = resultSet.getLong("column_length"); + long columnScale = resultSet.getLong("column_scale"); + String columnComment = resultSet.getString("column_comment"); + Object defaultValue = resultSet.getObject("default_value"); + boolean isNullable = resultSet.getString("is_nullable").equals("YES"); + + if (defaultValue != null && defaultValue.toString().contains("regclass")) + defaultValue = null; + + SeaTunnelDataType type = fromJdbcType(typeName, columnLength, columnScale); + long bitLen = 0; + switch (typeName) { + case PG_BYTEA: + bitLen = -1; + break; + case PG_TEXT: + columnLength = -1; + break; + case PG_INTERVAL: + columnLength = 50; + break; + case PG_GEOMETRY: + case PG_GEOGRAPHY: + columnLength = 255; + break; + case PG_BIT: + bitLen = columnLength; + break; + case PG_CHAR: + case PG_CHARACTER: + case PG_CHARACTER_VARYING: + default: + break; + } + + PhysicalColumn physicalColumn = + PhysicalColumn.of( + columnName, + type, + 0, + isNullable, + defaultValue, + columnComment, + fullTypeName, + false, + false, + bitLen, + null, + columnLength); + builder.column(physicalColumn); + } + + @Override + protected boolean createTableInternal(TablePath tablePath, CatalogTable table) + throws CatalogException { + String createTableSql = new PostgresCreateTableSqlBuilder(table).build(tablePath); + String dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); + Connection conn = getConnection(dbUrl); + log.info("create table sql: {}", createTableSql); + try (PreparedStatement ps = conn.prepareStatement(createTableSql)) { + ps.execute(); + } catch (Exception e) { + throw new CatalogException( + String.format("Failed creating table %s", tablePath.getFullName()), e); + } + return true; + } + + @Override + protected boolean dropTableInternal(TablePath tablePath) throws CatalogException { + String dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); + + String schemaName = tablePath.getSchemaName(); + String tableName = tablePath.getTableName(); + + String sql = "DROP TABLE IF EXISTS \"" + schemaName + "\".\"" + tableName + "\""; + Connection connection = getConnection(dbUrl); + try (PreparedStatement ps = connection.prepareStatement(sql)) { + // Will there exist concurrent drop for one table? + return ps.execute(); + } catch (SQLException e) { + throw new CatalogException( + String.format("Failed dropping table %s", tablePath.getFullName()), e); + } + } + + @Override + protected boolean createDatabaseInternal(String databaseName) throws CatalogException { + String sql = "CREATE DATABASE \"" + databaseName + "\""; + try (PreparedStatement ps = defaultConnection.prepareStatement(sql)) { + return ps.execute(); + } catch (Exception e) { + throw new CatalogException( + String.format( + "Failed creating database %s in catalog %s", + databaseName, this.catalogName), + e); + } + } + + @Override + public boolean tableExists(TablePath tablePath) throws CatalogException { + try { + return databaseExists(tablePath.getDatabaseName()) + && listTables(tablePath.getDatabaseName()) + .contains(tablePath.getSchemaAndTableName()); + } catch (DatabaseNotExistException e) { + return false; + } + } + + @Override + protected boolean dropDatabaseInternal(String databaseName) throws CatalogException { + String sql = "DROP DATABASE IF EXISTS \"" + databaseName + "\""; + try (PreparedStatement ps = defaultConnection.prepareStatement(sql)) { + return ps.execute(); + } catch (Exception e) { + throw new CatalogException( + String.format( + "Failed dropping database %s in catalog %s", + databaseName, this.catalogName), + e); + } + } + + /** + * @see MysqlType + * @see ResultSetImpl#getObjectStoredProc(int, int) + */ + @SuppressWarnings("unchecked") + private SeaTunnelDataType fromJdbcType(ResultSetMetaData metadata, int colIndex) + throws SQLException { + String columnTypeName = metadata.getColumnTypeName(colIndex); + Map dataTypeProperties = new HashMap<>(); + dataTypeProperties.put( + PostgresDataTypeConvertor.PRECISION, metadata.getPrecision(colIndex)); + dataTypeProperties.put(PostgresDataTypeConvertor.SCALE, metadata.getScale(colIndex)); + return new PostgresDataTypeConvertor().toSeaTunnelType(columnTypeName, dataTypeProperties); + } + + private SeaTunnelDataType fromJdbcType(String typeName, long precision, long scale) { + Map dataTypeProperties = new HashMap<>(); + dataTypeProperties.put(PostgresDataTypeConvertor.PRECISION, precision); + dataTypeProperties.put(PostgresDataTypeConvertor.SCALE, scale); + return new PostgresDataTypeConvertor().toSeaTunnelType(typeName, dataTypeProperties); + } + + @SuppressWarnings("MagicNumber") + private Map buildConnectorOptions(TablePath tablePath) { + Map options = new HashMap<>(8); + options.put("connector", "jdbc"); + options.put("url", baseUrl + tablePath.getDatabaseName()); + options.put("table-name", tablePath.getFullName()); + options.put("username", username); + options.put("password", pwd); + return options; + } + + private String getUrlFromDatabaseName(String databaseName) { + String url = baseUrl.endsWith("/") ? baseUrl : baseUrl + "/"; + return url + databaseName + suffix; + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogFactory.java new file mode 100644 index 000000000000..4db852960ed3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogFactory.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.configuration.util.OptionValidationException; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.factory.CatalogFactory; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; + +import com.google.auto.service.AutoService; + +import java.util.Optional; + +@AutoService(Factory.class) +public class PostgresCatalogFactory implements CatalogFactory { + public static final String IDENTIFIER = "Postgres"; + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Catalog createCatalog(String catalogName, ReadonlyConfig options) { + String urlWithDatabase = options.get(JdbcCatalogOptions.BASE_URL); + JdbcUrlUtil.UrlInfo urlInfo = JdbcUrlUtil.getUrlInfo(urlWithDatabase); + Optional defaultDatabase = urlInfo.getDefaultDatabase(); + if (!defaultDatabase.isPresent()) { + throw new OptionValidationException(JdbcCatalogOptions.BASE_URL); + } + return new PostgresCatalog( + catalogName, + options.get(JdbcCatalogOptions.USERNAME), + options.get(JdbcCatalogOptions.PASSWORD), + urlInfo, + options.get(JdbcCatalogOptions.SCHEMA)); + } + + @Override + public OptionRule optionRule() { + return JdbcCatalogOptions.BASE_RULE.build(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilder.java new file mode 100644 index 000000000000..85f4468bef9e --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCreateTableSqlBuilder.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.SqlType; + +import org.apache.commons.lang3.StringUtils; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_BYTEA; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresDataTypeConvertor.PG_NUMERIC; + +public class PostgresCreateTableSqlBuilder { + private List columns; + private PrimaryKey primaryKey; + private PostgresDataTypeConvertor postgresDataTypeConvertor; + private String sourceCatalogName; + + public PostgresCreateTableSqlBuilder(CatalogTable catalogTable) { + this.columns = catalogTable.getTableSchema().getColumns(); + this.primaryKey = catalogTable.getTableSchema().getPrimaryKey(); + this.postgresDataTypeConvertor = new PostgresDataTypeConvertor(); + this.sourceCatalogName = catalogTable.getCatalogName(); + } + + public String build(TablePath tablePath) { + StringBuilder createTableSql = new StringBuilder(); + createTableSql + .append("CREATE TABLE IF NOT EXISTS ") + .append(tablePath.getSchemaAndTableName()) + .append(" (\n"); + + List columnSqls = + columns.stream().map(this::buildColumnSql).collect(Collectors.toList()); + + createTableSql.append(String.join(",\n", columnSqls)); + createTableSql.append("\n);"); + + List commentSqls = + columns.stream() + .filter(column -> StringUtils.isNotBlank(column.getComment())) + .map( + columns -> + buildColumnCommentSql( + columns, tablePath.getSchemaAndTableName())) + .collect(Collectors.toList()); + + if (!commentSqls.isEmpty()) { + createTableSql.append("\n"); + createTableSql.append(String.join(";\n", commentSqls)).append(";"); + } + + return createTableSql.toString(); + } + + private String buildColumnSql(Column column) { + StringBuilder columnSql = new StringBuilder(); + columnSql.append(column.getName()).append(" "); + + // For simplicity, assume the column type in SeaTunnelDataType is the same as in PostgreSQL + String columnType = + sourceCatalogName.equals("postgres") + ? column.getSourceType() + : buildColumnType(column); + columnSql.append(columnType); + + // Add NOT NULL if column is not nullable + if (!column.isNullable()) { + columnSql.append(" NOT NULL"); + } + + // Add primary key directly after the column if it is a primary key + if (primaryKey != null && primaryKey.getColumnNames().contains(column.getName())) { + columnSql.append(" PRIMARY KEY"); + } + + // Add default value if exists + // if (column.getDefaultValue() != null) { + // columnSql.append(" DEFAULT + // '").append(column.getDefaultValue().toString()).append("'"); + // } + + return columnSql.toString(); + } + + private String buildColumnType(Column column) { + SqlType sqlType = column.getDataType().getSqlType(); + Long columnLength = column.getLongColumnLength(); + switch (sqlType) { + case BYTES: + return PG_BYTEA; + case STRING: + if (columnLength > 0 && columnLength < 10485760) { + return "varchar(" + columnLength + ")"; + } else { + return "text"; + } + default: + String type = postgresDataTypeConvertor.toConnectorType(column.getDataType(), null); + if (type.equals(PG_NUMERIC)) { + DecimalType decimalType = (DecimalType) column.getDataType(); + return "numeric(" + + decimalType.getPrecision() + + "," + + decimalType.getScale() + + ")"; + } + return type; + } + } + + private String buildColumnCommentSql(Column column, String tableName) { + StringBuilder columnCommentSql = new StringBuilder(); + columnCommentSql.append("COMMENT ON COLUMN ").append(tableName).append("."); + columnCommentSql + .append(column.getName()) + .append(" IS '") + .append(column.getComment()) + .append("'"); + return columnCommentSql.toString(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresDataTypeConvertor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresDataTypeConvertor.java index 81bf5ca0665f..c87a2fc11884 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresDataTypeConvertor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresDataTypeConvertor.java @@ -65,41 +65,46 @@ public class PostgresDataTypeConvertor implements DataTypeConvertor { // float <=> float8 // boolean <=> bool // decimal <=> numeric - private static final String PG_SMALLSERIAL = "smallserial"; - private static final String PG_SERIAL = "serial"; - private static final String PG_BIGSERIAL = "bigserial"; - private static final String PG_BYTEA = "bytea"; - private static final String PG_BYTEA_ARRAY = "_bytea"; - private static final String PG_SMALLINT = "int2"; - private static final String PG_SMALLINT_ARRAY = "_int2"; - private static final String PG_INTEGER = "int4"; - private static final String PG_INTEGER_ARRAY = "_int4"; - private static final String PG_BIGINT = "int8"; - private static final String PG_BIGINT_ARRAY = "_int8"; - private static final String PG_REAL = "float4"; - private static final String PG_REAL_ARRAY = "_float4"; - private static final String PG_DOUBLE_PRECISION = "float8"; - private static final String PG_DOUBLE_PRECISION_ARRAY = "_float8"; - private static final String PG_NUMERIC = "numeric"; - private static final String PG_NUMERIC_ARRAY = "_numeric"; - private static final String PG_BOOLEAN = "bool"; - private static final String PG_BOOLEAN_ARRAY = "_bool"; - private static final String PG_TIMESTAMP = "timestamp"; - private static final String PG_TIMESTAMP_ARRAY = "_timestamp"; - private static final String PG_TIMESTAMPTZ = "timestamptz"; - private static final String PG_TIMESTAMPTZ_ARRAY = "_timestamptz"; - private static final String PG_DATE = "date"; - private static final String PG_DATE_ARRAY = "_date"; - private static final String PG_TIME = "time"; - private static final String PG_TIME_ARRAY = "_time"; - private static final String PG_TEXT = "text"; - private static final String PG_TEXT_ARRAY = "_text"; - private static final String PG_CHAR = "bpchar"; - private static final String PG_CHAR_ARRAY = "_bpchar"; - private static final String PG_CHARACTER = "character"; - private static final String PG_CHARACTER_ARRAY = "_character"; - private static final String PG_CHARACTER_VARYING = "varchar"; - private static final String PG_CHARACTER_VARYING_ARRAY = "_varchar"; + public static final String PG_SMALLSERIAL = "smallserial"; + public static final String PG_SERIAL = "serial"; + public static final String PG_BIGSERIAL = "bigserial"; + public static final String PG_BYTEA = "bytea"; + + public static final String PG_BIT = "bit"; + public static final String PG_BYTEA_ARRAY = "_bytea"; + public static final String PG_SMALLINT = "int2"; + public static final String PG_SMALLINT_ARRAY = "_int2"; + public static final String PG_INTEGER = "int4"; + public static final String PG_INTEGER_ARRAY = "_int4"; + public static final String PG_BIGINT = "int8"; + public static final String PG_BIGINT_ARRAY = "_int8"; + public static final String PG_REAL = "float4"; + public static final String PG_REAL_ARRAY = "_float4"; + public static final String PG_DOUBLE_PRECISION = "float8"; + public static final String PG_DOUBLE_PRECISION_ARRAY = "_float8"; + public static final String PG_NUMERIC = "numeric"; + public static final String PG_NUMERIC_ARRAY = "_numeric"; + public static final String PG_BOOLEAN = "bool"; + public static final String PG_BOOLEAN_ARRAY = "_bool"; + public static final String PG_TIMESTAMP = "timestamp"; + public static final String PG_TIMESTAMP_ARRAY = "_timestamp"; + public static final String PG_TIMESTAMPTZ = "timestamptz"; + public static final String PG_TIMESTAMPTZ_ARRAY = "_timestamptz"; + public static final String PG_DATE = "date"; + public static final String PG_DATE_ARRAY = "_date"; + public static final String PG_TIME = "time"; + public static final String PG_TIME_ARRAY = "_time"; + public static final String PG_TEXT = "text"; + public static final String PG_TEXT_ARRAY = "_text"; + public static final String PG_CHAR = "bpchar"; + public static final String PG_CHAR_ARRAY = "_bpchar"; + public static final String PG_CHARACTER = "character"; + public static final String PG_CHARACTER_ARRAY = "_character"; + public static final String PG_CHARACTER_VARYING = "varchar"; + public static final String PG_CHARACTER_VARYING_ARRAY = "_varchar"; + public static final String PG_INTERVAL = "interval"; + public static final String PG_GEOMETRY = "geometry"; + public static final String PG_GEOGRAPHY = "geography"; @Override public SeaTunnelDataType toSeaTunnelType(String connectorDataType) { @@ -117,6 +122,7 @@ public SeaTunnelDataType toSeaTunnelType( case PG_BOOLEAN_ARRAY: return ArrayType.BOOLEAN_ARRAY_TYPE; case PG_BYTEA: + case PG_BIT: return PrimitiveByteArrayType.INSTANCE; case PG_BYTEA_ARRAY: return ArrayType.BYTE_ARRAY_TYPE; @@ -151,6 +157,9 @@ public SeaTunnelDataType toSeaTunnelType( case PG_CHARACTER: case PG_CHARACTER_VARYING: case PG_TEXT: + case PG_INTERVAL: + case PG_GEOMETRY: + case PG_GEOGRAPHY: return BasicType.STRING_TYPE; case PG_CHAR_ARRAY: case PG_CHARACTER_ARRAY: @@ -158,6 +167,7 @@ public SeaTunnelDataType toSeaTunnelType( case PG_TEXT_ARRAY: return ArrayType.STRING_ARRAY_TYPE; case PG_TIMESTAMP: + case PG_TIMESTAMPTZ: return LocalTimeType.LOCAL_DATE_TIME_TYPE; case PG_TIME: return LocalTimeType.LOCAL_TIME_TYPE; @@ -166,7 +176,6 @@ public SeaTunnelDataType toSeaTunnelType( case PG_TIMESTAMP_ARRAY: case PG_NUMERIC_ARRAY: - case PG_TIMESTAMPTZ: case PG_TIMESTAMPTZ_ARRAY: case PG_TIME_ARRAY: case PG_DATE_ARRAY: diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java index f376f47af11b..ea04c60bff56 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalog.java @@ -32,8 +32,11 @@ import org.apache.seatunnel.common.utils.JdbcUrlUtil; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.AbstractJdbcCatalog; +import org.apache.commons.lang3.StringUtils; import org.apache.commons.lang3.tuple.Pair; +import lombok.extern.slf4j.Slf4j; + import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.DriverManager; @@ -50,6 +53,7 @@ import java.util.Optional; import java.util.Set; +@Slf4j public class SqlServerCatalog extends AbstractJdbcCatalog { private static final Set SYS_DATABASES = new HashSet<>(4); @@ -62,8 +66,12 @@ public class SqlServerCatalog extends AbstractJdbcCatalog { } public SqlServerCatalog( - String catalogName, String username, String pwd, JdbcUrlUtil.UrlInfo urlInfo) { - super(catalogName, username, pwd, urlInfo); + String catalogName, + String username, + String pwd, + JdbcUrlUtil.UrlInfo urlInfo, + String defaultSchema) { + super(catalogName, username, pwd, urlInfo, defaultSchema); } @Override @@ -135,6 +143,15 @@ public CatalogTable getTable(TablePath tablePath) if (!tableExists(tablePath)) { throw new TableNotExistException(catalogName, tablePath); } + String tableSql = + StringUtils.isNotEmpty(tablePath.getTableName()) + ? "AND tbl.name = '" + tablePath.getTableName() + "'" + : ""; + + String columnSql = + String.format( + " SELECT tbl.name AS table_name, \n col.name AS column_name, \n ext.value AS comment, \n col.column_id AS column_id, \n types.name AS type, \n col.max_length AS max_length, \n col.precision AS precision, \n col.scale AS scale, \n col.is_nullable AS is_nullable, \n def.definition AS default_value\n FROM sys.tables tbl \nINNER JOIN sys.columns col \n ON tbl.object_id = col.object_id \n LEFT JOIN sys.types types \n ON col.user_type_id = types.user_type_id \n LEFT JOIN sys.extended_properties ext \n ON ext.major_id = col.object_id and ext.minor_id = col.column_id \n LEFT JOIN sys.default_constraints def ON col.default_object_id = def.object_id \n AND ext.minor_id = col.column_id \n AND ext.name = 'MS_Description' \n WHERE schema_name(tbl.schema_id) = '%s' \n %s \n ORDER BY tbl.name, col.column_id", + tablePath.getSchemaName(), tableSql); String dbUrl = getUrlFromDatabaseName(tablePath.getDatabaseName()); try (Connection conn = DriverManager.getConnection(dbUrl, username, pwd)) { @@ -152,40 +169,13 @@ public CatalogTable getTable(TablePath tablePath) tablePath.getSchemaName(), tablePath.getTableName()); - try (PreparedStatement ps = - conn.prepareStatement( - String.format( - "SELECT * FROM %s WHERE 1 = 0;", - tablePath.getFullNameWithQuoted("\"")))) { - ResultSetMetaData tableMetaData = ps.getMetaData(); + try (PreparedStatement ps = conn.prepareStatement(columnSql); + ResultSet resultSet = ps.executeQuery(); ) { TableSchema.Builder builder = TableSchema.builder(); - // add column - for (int i = 1; i <= tableMetaData.getColumnCount(); i++) { - String columnName = tableMetaData.getColumnName(i); - SeaTunnelDataType type = fromJdbcType(tableMetaData, i); - int columnDisplaySize = tableMetaData.getColumnDisplaySize(i); - String comment = tableMetaData.getColumnLabel(i); - boolean isNullable = - tableMetaData.isNullable(i) == ResultSetMetaData.columnNullable; - Object defaultValue = - getColumnDefaultValue( - metaData, - tablePath.getDatabaseName(), - tablePath.getSchemaName(), - tablePath.getTableName(), - columnName) - .orElse(null); - - PhysicalColumn physicalColumn = - PhysicalColumn.of( - columnName, - type, - columnDisplaySize, - isNullable, - defaultValue, - comment); - builder.column(physicalColumn); + while (resultSet.next()) { + buildTable(resultSet, builder); } + // add primary key primaryKey.ifPresent(builder::primaryKey); // add constraint key @@ -201,7 +191,8 @@ public CatalogTable getTable(TablePath tablePath) builder.build(), buildConnectorOptions(tablePath), Collections.emptyList(), - ""); + "", + "sqlserver"); } } catch (Exception e) { @@ -210,10 +201,111 @@ public CatalogTable getTable(TablePath tablePath) } } + private void buildTable(ResultSet resultSet, TableSchema.Builder builder) throws SQLException { + String columnName = resultSet.getString("column_name"); + String sourceType = resultSet.getString("type"); + // String typeName = resultSet.getString("DATA_TYPE").toUpperCase(); + int precision = resultSet.getInt("precision"); + int scale = resultSet.getInt("scale"); + long columnLength = resultSet.getLong("max_length"); + SeaTunnelDataType type = fromJdbcType(sourceType, precision, scale); + String comment = resultSet.getString("comment"); + Object defaultValue = resultSet.getObject("default_value"); + if (defaultValue != null) { + defaultValue = + defaultValue.toString().replace("(", "").replace("'", "").replace(")", ""); + } + boolean isNullable = resultSet.getBoolean("is_nullable"); + long bitLen = 0; + StringBuilder sb = new StringBuilder(sourceType); + Pair> parse = SqlServerType.parse(sourceType); + switch (parse.getLeft()) { + case BINARY: + case VARBINARY: + // Uniform conversion to bits + if (columnLength != -1) { + bitLen = columnLength * 4 * 8; + sourceType = sb.append("(").append(columnLength).append(")").toString(); + } else { + sourceType = sb.append("(").append("max").append(")").toString(); + bitLen = columnLength; + } + break; + case TIMESTAMP: + bitLen = columnLength << 3; + break; + case VARCHAR: + case NCHAR: + case NVARCHAR: + case CHAR: + if (columnLength != -1) { + sourceType = sb.append("(").append(columnLength).append(")").toString(); + } else { + sourceType = sb.append("(").append("max").append(")").toString(); + } + break; + case DECIMAL: + case NUMERIC: + sourceType = + sb.append("(") + .append(precision) + .append(",") + .append(scale) + .append(")") + .toString(); + break; + case TEXT: + columnLength = Integer.MAX_VALUE; + break; + case NTEXT: + columnLength = Integer.MAX_VALUE >> 1; + break; + case IMAGE: + bitLen = Integer.MAX_VALUE * 8L; + break; + default: + break; + } + PhysicalColumn physicalColumn = + PhysicalColumn.of( + columnName, + type, + 0, + isNullable, + defaultValue, + comment, + sourceType, + false, + false, + bitLen, + null, + columnLength); + builder.column(physicalColumn); + } + + private SeaTunnelDataType fromJdbcType(String typeName, int precision, int scale) { + Pair> pair = SqlServerType.parse(typeName); + Map dataTypeProperties = new HashMap<>(); + dataTypeProperties.put(SqlServerDataTypeConvertor.PRECISION, precision); + dataTypeProperties.put(SqlServerDataTypeConvertor.SCALE, scale); + return new SqlServerDataTypeConvertor().toSeaTunnelType(pair.getLeft(), dataTypeProperties); + } + @Override protected boolean createTableInternal(TablePath tablePath, CatalogTable table) throws CatalogException { - throw new UnsupportedOperationException("Unsupported create table"); + + String createTableSql = + SqlServerCreateTableSqlBuilder.builder(tablePath, table).build(tablePath, table); + log.info("create table sql: {}", createTableSql); + try (Connection conn = DriverManager.getConnection(defaultUrl, username, pwd); + PreparedStatement ps = conn.prepareStatement(createTableSql)) { + System.out.println(createTableSql); + return ps.execute(); + } catch (Exception e) { + throw new CatalogException( + String.format("Failed creating table %s", tablePath.getFullName()), e); + } } @Override @@ -222,7 +314,8 @@ protected boolean dropTableInternal(TablePath tablePath) throws CatalogException try (Connection conn = DriverManager.getConnection(dbUrl, username, pwd); PreparedStatement ps = conn.prepareStatement( - String.format("DROP TABLE IF EXIST %s", tablePath.getFullName()))) { + String.format( + "DROP TABLE IF EXISTS %s", tablePath.getFullName()))) { // Will there exist concurrent drop for one table? return ps.execute(); } catch (SQLException e) { @@ -289,4 +382,9 @@ private Map buildConnectorOptions(TablePath tablePath) { private String getUrlFromDatabaseName(String databaseName) { return baseUrl + ";databaseName=" + databaseName + ";" + suffix; } + + private String getCreateTableSql(TablePath tablePath, CatalogTable table) { + + return ""; + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogFactory.java index a59b7e399f38..9ddd035b2ad5 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogFactory.java @@ -32,10 +32,11 @@ @AutoService(Factory.class) public class SqlServerCatalogFactory implements CatalogFactory { + public static final String IDENTIFIER = "SqlServer"; @Override public String factoryIdentifier() { - return "SqlServer"; + return IDENTIFIER; } @Override @@ -50,7 +51,8 @@ public Catalog createCatalog(String catalogName, ReadonlyConfig options) { catalogName, options.get(JdbcCatalogOptions.USERNAME), options.get(JdbcCatalogOptions.PASSWORD), - urlInfo); + urlInfo, + options.get(JdbcCatalogOptions.SCHEMA)); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilder.java new file mode 100644 index 000000000000..cf100075ad1d --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCreateTableSqlBuilder.java @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.Column; +import org.apache.seatunnel.api.table.catalog.ConstraintKey; +import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.catalog.TableSchema; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.SqlType; + +import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +public class SqlServerCreateTableSqlBuilder { + + private final String tableName; + private List columns; + + private String comment; + + private String engine; + private String charset; + private String collate; + + private PrimaryKey primaryKey; + + private List constraintKeys; + + private SqlServerDataTypeConvertor sqlServerDataTypeConvertor; + + private SqlServerCreateTableSqlBuilder(String tableName) { + checkNotNull(tableName, "tableName must not be null"); + this.tableName = tableName; + this.sqlServerDataTypeConvertor = new SqlServerDataTypeConvertor(); + } + + public static SqlServerCreateTableSqlBuilder builder( + TablePath tablePath, CatalogTable catalogTable) { + checkNotNull(tablePath, "tablePath must not be null"); + checkNotNull(catalogTable, "catalogTable must not be null"); + + TableSchema tableSchema = catalogTable.getTableSchema(); + checkNotNull(tableSchema, "tableSchema must not be null"); + + return new SqlServerCreateTableSqlBuilder(tablePath.getTableName()) + .comment(catalogTable.getComment()) + // todo: set charset and collate + .engine(null) + .charset(null) + .primaryKey(tableSchema.getPrimaryKey()) + .constraintKeys(tableSchema.getConstraintKeys()) + .addColumn(tableSchema.getColumns()); + } + + public SqlServerCreateTableSqlBuilder addColumn(List columns) { + checkArgument(CollectionUtils.isNotEmpty(columns), "columns must not be empty"); + this.columns = columns; + return this; + } + + public SqlServerCreateTableSqlBuilder primaryKey(PrimaryKey primaryKey) { + this.primaryKey = primaryKey; + return this; + } + + public SqlServerCreateTableSqlBuilder constraintKeys(List constraintKeys) { + this.constraintKeys = constraintKeys; + return this; + } + + public SqlServerCreateTableSqlBuilder engine(String engine) { + this.engine = engine; + return this; + } + + public SqlServerCreateTableSqlBuilder charset(String charset) { + this.charset = charset; + return this; + } + + public SqlServerCreateTableSqlBuilder collate(String collate) { + this.collate = collate; + return this; + } + + public SqlServerCreateTableSqlBuilder comment(String comment) { + this.comment = comment; + return this; + } + + public String build(TablePath tablePath, CatalogTable catalogTable) { + List sqls = new ArrayList<>(); + String sqlTableName = tablePath.getFullName(); + Map columnComments = new HashMap<>(); + sqls.add( + String.format( + "IF OBJECT_ID('%s', 'U') IS NULL \n" + + "BEGIN \n" + + "CREATE TABLE %s ( \n%s\n)", + sqlTableName, + sqlTableName, + buildColumnsIdentifySql(catalogTable.getCatalogName(), columnComments))); + if (engine != null) { + sqls.add("ENGINE = " + engine); + } + if (charset != null) { + sqls.add("DEFAULT CHARSET = " + charset); + } + if (collate != null) { + sqls.add("COLLATE = " + collate); + } + String sqlTableSql = String.join(" ", sqls) + ";"; + StringBuilder tableAndColumnComment = new StringBuilder(); + if (comment != null) { + sqls.add("COMMENT = '" + comment + "'"); + tableAndColumnComment.append( + String.format( + "EXEC %s.sys.sp_addextendedproperty 'MS_Description', N'%s', 'schema', N'%s', 'table', N'%s';\n", + tablePath.getDatabaseName(), + comment, + tablePath.getSchemaName(), + tablePath.getTableName())); + } + String columnComment = + "EXEC %s.sys.sp_addextendedproperty 'MS_Description', N'%s', 'schema', N'%s', 'table', N'%s', 'column', N'%s';\n"; + columnComments.forEach( + (fieldName, com) -> { + tableAndColumnComment.append( + String.format( + columnComment, + tablePath.getDatabaseName(), + com, + tablePath.getSchemaName(), + tablePath.getTableName(), + fieldName)); + }); + return String.join("\n", sqlTableSql, tableAndColumnComment.toString(), "END"); + } + + private String buildColumnsIdentifySql(String catalogName, Map columnComments) { + List columnSqls = new ArrayList<>(); + for (Column column : columns) { + columnSqls.add("\t" + buildColumnIdentifySql(column, catalogName, columnComments)); + } + if (primaryKey != null) { + columnSqls.add("\t" + buildPrimaryKeySql()); + } + if (CollectionUtils.isNotEmpty(constraintKeys)) { + for (ConstraintKey constraintKey : constraintKeys) { + if (StringUtils.isBlank(constraintKey.getConstraintName())) { + continue; + } + } + } + return String.join(", \n", columnSqls); + } + + private String buildColumnIdentifySql( + Column column, String catalogName, Map columnComments) { + final List columnSqls = new ArrayList<>(); + columnSqls.add(column.getName()); + String tyNameDef = ""; + if (StringUtils.equals(catalogName, "sqlserver")) { + columnSqls.add(column.getSourceType()); + } else { + // Column name + SqlType dataType = column.getDataType().getSqlType(); + boolean isBytes = StringUtils.equals(dataType.name(), SqlType.BYTES.name()); + Long columnLength = column.getLongColumnLength(); + Long bitLen = column.getBitLen(); + bitLen = bitLen == -1 || bitLen <= 8 ? bitLen : bitLen >> 3; + if (isBytes) { + if (bitLen > 8000 || bitLen == -1) { + columnSqls.add(SqlServerType.VARBINARY.getName()); + } else { + columnSqls.add(SqlServerType.BINARY.getName()); + tyNameDef = SqlServerType.BINARY.getName(); + } + columnSqls.add("(" + (bitLen == -1 || bitLen > 8000 ? "max)" : bitLen + ")")); + } else { + // Add column type + SqlServerType sqlServerType = + sqlServerDataTypeConvertor.toConnectorType(column.getDataType(), null); + String typeName = sqlServerType.getName(); + String fieldSuffixSql = null; + tyNameDef = typeName; + // Add column length + if (StringUtils.equals(SqlServerType.VARCHAR.getName(), typeName)) { + if (columnLength > 8000 || columnLength == -1) { + columnSqls.add(typeName); + fieldSuffixSql = "(max)"; + } else if (columnLength > 4000) { + columnSqls.add(SqlServerType.VARCHAR.getName()); + fieldSuffixSql = "(" + columnLength + ")"; + } else { + columnSqls.add(SqlServerType.NVARCHAR.getName()); + if (columnLength > 0) { + fieldSuffixSql = "(" + columnLength + ")"; + } + } + columnSqls.add(fieldSuffixSql); + } else if (StringUtils.equals(SqlServerType.DECIMAL.getName(), typeName)) { + columnSqls.add(typeName); + DecimalType decimalType = (DecimalType) column.getDataType(); + columnSqls.add( + String.format( + "(%d, %d)", + decimalType.getPrecision(), decimalType.getScale())); + } else { + columnSqls.add(typeName); + } + } + } + // nullable + if (column.isNullable()) { + columnSqls.add("NULL"); + } else { + columnSqls.add("NOT NULL"); + } + // default value + // if (column.getDefaultValue() != null) { + // String defaultValue = "'" + column.getDefaultValue().toString() + "'"; + // if (StringUtils.equals(SqlServerType.BINARY.getName(), tyNameDef) + // && defaultValue.contains("b'")) { + // String rep = defaultValue.replace("b", "").replace("'", ""); + // defaultValue = "0x" + Integer.toHexString(Integer.parseInt(rep)); + // } else if (StringUtils.equals(SqlServerType.BIT.getName(), tyNameDef) + // && defaultValue.contains("b'")) { + // defaultValue = defaultValue.replace("b", "").replace("'", ""); + // } + // columnSqls.add("DEFAULT " + defaultValue); + // } + // comment + if (column.getComment() != null) { + columnComments.put(column.getName(), column.getComment()); + } + + return String.join(" ", columnSqls); + } + + private String buildPrimaryKeySql() { + // .map(columnName -> "`" + columnName + "`") + String key = String.join(", ", primaryKey.getColumnNames()); + // add sort type + return String.format("PRIMARY KEY (%s)", key); + } + + private String buildConstraintKeySql(ConstraintKey constraintKey) { + ConstraintKey.ConstraintType constraintType = constraintKey.getConstraintType(); + String indexColumns = + constraintKey.getColumnNames().stream() + .map( + constraintKeyColumn -> { + if (constraintKeyColumn.getSortType() == null) { + return String.format( + "`%s`", constraintKeyColumn.getColumnName()); + } + return String.format( + "`%s` %s", + constraintKeyColumn.getColumnName(), + constraintKeyColumn.getSortType().name()); + }) + .collect(Collectors.joining(", ")); + String keyName = null; + switch (constraintType) { + case KEY: + keyName = "KEY"; + break; + case UNIQUE_KEY: + keyName = "UNIQUE KEY"; + break; + case FOREIGN_KEY: + keyName = "FOREIGN KEY"; + // todo: + break; + default: + throw new UnsupportedOperationException( + "Unsupported constraint type: " + constraintType); + } + return String.format( + "%s `%s` (%s)", keyName, constraintKey.getConstraintName(), indexColumns); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerDataTypeConvertor.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerDataTypeConvertor.java index e04be54a56b3..afad20c67c14 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerDataTypeConvertor.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerDataTypeConvertor.java @@ -132,9 +132,9 @@ public SqlServerType toConnectorType( case DATE: return SqlServerType.DATE; case TIME: - return SqlServerType.DATETIME; + return SqlServerType.TIME; case TIMESTAMP: - return SqlServerType.TIMESTAMP; + return SqlServerType.DATETIME2; default: throw new JdbcConnectorException( CommonErrorCode.UNSUPPORTED_DATA_TYPE, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerURLParser.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerURLParser.java index 94b0bde5abfd..fa8ed1869d47 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerURLParser.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerURLParser.java @@ -77,12 +77,16 @@ public static JdbcUrlUtil.UrlInfo parse(String url) { String suffix = props.entrySet().stream() + .filter( + e -> + !e.getKey().equals("databaseName") + && !e.getKey().equals("database")) .map(e -> e.getKey() + "=" + e.getValue()) - .collect(Collectors.joining(";", ";", "")); + .collect(Collectors.joining(";", "", "")); suffix = Optional.ofNullable(suffix).orElse(""); return new JdbcUrlUtil.UrlInfo( url, - String.format("jdbc:sqlserver://%s:%s", serverName, port) + suffix, + String.format("jdbc:sqlserver://%s:%s", serverName, port) + ";" + suffix, serverName, port, dbInstance, 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 24ae0580f325..f5d1613c53ef 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 @@ -74,7 +74,7 @@ public interface JdbcOptions { Option BATCH_INTERVAL_MS = Options.key("batch_interval_ms") .intType() - .defaultValue(1000) + .defaultValue(0) .withDescription("batch interval milliSecond"); Option IS_EXACTLY_ONCE = @@ -122,6 +122,23 @@ public interface JdbcOptions { .defaultValue(false) .withDescription("support upsert by query primary_key exist"); + Option ENABLE_UPSERT = + Options.key("enable_upsert") + .booleanType() + .defaultValue(true) + .withDescription("enable upsert by primary_keys exist"); + Option IS_PRIMARY_KEY_UPDATED = + Options.key("is_primary_key_updated") + .booleanType() + .defaultValue(true) + .withDescription( + "is the primary key updated when performing an update operation"); + Option SUPPORT_UPSERT_BY_INSERT_ONLY = + Options.key("support_upsert_by_insert_only") + .booleanType() + .defaultValue(false) + .withDescription("support upsert by insert only"); + /** source config */ Option PARTITION_COLUMN = Options.key("partition_column") diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSinkConfig.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSinkConfig.java index f7a3cd29109b..af24a9a6b03a 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSinkConfig.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/config/JdbcSinkConfig.java @@ -25,10 +25,12 @@ import java.io.Serializable; import java.util.List; -import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.SUPPORT_UPSERT_BY_QUERY_PRIMARY_KEY_EXIST; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.ENABLE_UPSERT; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.IS_PRIMARY_KEY_UPDATED; +import static org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcOptions.SUPPORT_UPSERT_BY_INSERT_ONLY; @Data -@Builder(builderClassName = "Builder") +@Builder public class JdbcSinkConfig implements Serializable { private static final long serialVersionUID = 2L; @@ -38,17 +40,21 @@ public class JdbcSinkConfig implements Serializable { private String database; private String table; private List primaryKeys; - private boolean supportUpsertByQueryPrimaryKeyExist; + private boolean enableUpsert; + @Builder.Default private boolean isPrimaryKeyUpdated = true; + private boolean supportUpsertByInsertOnly; public static JdbcSinkConfig of(ReadonlyConfig config) { - JdbcSinkConfig.Builder builder = JdbcSinkConfig.builder(); + JdbcSinkConfigBuilder builder = JdbcSinkConfig.builder(); builder.jdbcConnectionConfig(JdbcConnectionConfig.of(config)); builder.isExactlyOnce(config.get(JdbcOptions.IS_EXACTLY_ONCE)); config.getOptional(JdbcOptions.PRIMARY_KEYS).ifPresent(builder::primaryKeys); config.getOptional(JdbcOptions.DATABASE).ifPresent(builder::database); config.getOptional(JdbcOptions.TABLE).ifPresent(builder::table); - config.getOptional(SUPPORT_UPSERT_BY_QUERY_PRIMARY_KEY_EXIST) - .ifPresent(builder::supportUpsertByQueryPrimaryKeyExist); + config.getOptional(ENABLE_UPSERT).ifPresent(builder::enableUpsert); + config.getOptional(IS_PRIMARY_KEY_UPDATED).ifPresent(builder::isPrimaryKeyUpdated); + config.getOptional(SUPPORT_UPSERT_BY_INSERT_ONLY) + .ifPresent(builder::supportUpsertByInsertOnly); config.getOptional(JdbcOptions.QUERY).ifPresent(builder::simpleSql); return builder.build(); } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/JdbcOutputFormatBuilder.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/JdbcOutputFormatBuilder.java index 78e8814392f3..cd752d439608 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/JdbcOutputFormatBuilder.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/JdbcOutputFormatBuilder.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; @@ -55,7 +56,11 @@ public JdbcOutputFormat build() { JdbcOutputFormat.StatementExecutorFactory statementExecutorFactory; final String database = jdbcSinkConfig.getDatabase(); - final String table = jdbcSinkConfig.getTable(); + final String table = + dialect.extractTableName( + TablePath.of( + jdbcSinkConfig.getDatabase() + "." + jdbcSinkConfig.getTable())); + final List primaryKeys = jdbcSinkConfig.getPrimaryKeys(); if (StringUtils.isNotBlank(jdbcSinkConfig.getSimpleSql())) { statementExecutorFactory = @@ -76,7 +81,9 @@ public JdbcOutputFormat build() { table, seaTunnelRowType, primaryKeys.toArray(new String[0]), - jdbcSinkConfig.isSupportUpsertByQueryPrimaryKeyExist()); + jdbcSinkConfig.isEnableUpsert(), + jdbcSinkConfig.isPrimaryKeyUpdated(), + jdbcSinkConfig.isSupportUpsertByInsertOnly()); } return new JdbcOutputFormat( @@ -104,7 +111,9 @@ private static JdbcBatchStatementExecutor createUpsertBufferedExec String table, SeaTunnelRowType rowType, String[] pkNames, - boolean supportUpsertByQueryPrimaryKeyExist) { + boolean enableUpsert, + boolean isPrimaryKeyUpdated, + boolean supportUpsertByInsertOnly) { int[] pkFields = Arrays.stream(pkNames).mapToInt(rowType::indexOf).toArray(); SeaTunnelDataType[] pkTypes = Arrays.stream(pkFields) @@ -123,7 +132,9 @@ private static JdbcBatchStatementExecutor createUpsertBufferedExec pkNames, pkTypes, keyExtractor, - supportUpsertByQueryPrimaryKeyExist); + enableUpsert, + isPrimaryKeyUpdated, + supportUpsertByInsertOnly); return new BufferReducedBatchStatementExecutor( upsertExecutor, deleteExecutor, keyExtractor, Function.identity()); } @@ -136,17 +147,44 @@ private static JdbcBatchStatementExecutor createUpsertExecutor( String[] pkNames, SeaTunnelDataType[] pkTypes, Function keyExtractor, - boolean supportUpsertByQueryPrimaryKeyExist) { - Optional upsertSQL = - dialect.getUpsertStatement(database, table, rowType.getFieldNames(), pkNames); - if (upsertSQL.isPresent()) { - return createSimpleExecutor(upsertSQL.get(), rowType, dialect.getRowConverter()); + boolean enableUpsert, + boolean isPrimaryKeyUpdated, + boolean supportUpsertByInsertOnly) { + if (supportUpsertByInsertOnly) { + return createInsertOnlyExecutor(dialect, database, table, rowType); } - if (supportUpsertByQueryPrimaryKeyExist) { + if (enableUpsert) { + Optional upsertSQL = + dialect.getUpsertStatement(database, table, rowType.getFieldNames(), pkNames); + if (upsertSQL.isPresent()) { + return createSimpleExecutor(upsertSQL.get(), rowType, dialect.getRowConverter()); + } return createInsertOrUpdateByQueryExecutor( - dialect, database, table, rowType, pkNames, pkTypes, keyExtractor); + dialect, + database, + table, + rowType, + pkNames, + pkTypes, + keyExtractor, + isPrimaryKeyUpdated); } - return createInsertOrUpdateExecutor(dialect, database, table, rowType, pkNames); + return createInsertOrUpdateExecutor( + dialect, database, table, rowType, pkNames, isPrimaryKeyUpdated); + } + + private static JdbcBatchStatementExecutor createInsertOnlyExecutor( + JdbcDialect dialect, String database, String table, SeaTunnelRowType rowType) { + + return new SimpleBatchStatementExecutor( + connection -> + FieldNamedPreparedStatement.prepareStatement( + connection, + dialect.getInsertIntoStatement( + database, table, rowType.getFieldNames()), + rowType.getFieldNames()), + rowType, + dialect.getRowConverter()); } private static JdbcBatchStatementExecutor createInsertOrUpdateExecutor( @@ -154,7 +192,8 @@ private static JdbcBatchStatementExecutor createInsertOrUpdateExec String database, String table, SeaTunnelRowType rowType, - String[] pkNames) { + String[] pkNames, + boolean isPrimaryKeyUpdated) { return new InsertOrUpdateBatchStatementExecutor( connection -> @@ -167,7 +206,11 @@ private static JdbcBatchStatementExecutor createInsertOrUpdateExec FieldNamedPreparedStatement.prepareStatement( connection, dialect.getUpdateStatement( - database, table, rowType.getFieldNames(), pkNames), + database, + table, + rowType.getFieldNames(), + pkNames, + isPrimaryKeyUpdated), rowType.getFieldNames()), rowType, dialect.getRowConverter()); @@ -180,7 +223,8 @@ private static JdbcBatchStatementExecutor createInsertOrUpdateByQu SeaTunnelRowType rowType, String[] pkNames, SeaTunnelDataType[] pkTypes, - Function keyExtractor) { + Function keyExtractor, + boolean isPrimaryKeyUpdated) { SeaTunnelRowType keyRowType = new SeaTunnelRowType(pkNames, pkTypes); return new InsertOrUpdateBatchStatementExecutor( connection -> @@ -198,7 +242,11 @@ private static JdbcBatchStatementExecutor createInsertOrUpdateByQu FieldNamedPreparedStatement.prepareStatement( connection, dialect.getUpdateStatement( - database, table, rowType.getFieldNames(), pkNames), + database, + table, + rowType.getFieldNames(), + pkNames, + isPrimaryKeyUpdated), rowType.getFieldNames()), keyRowType, keyExtractor, diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java index e8967fce08fb..8a0b31a5eeb9 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/JdbcDialect.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; @@ -109,7 +110,21 @@ default String getInsertIntoStatement(String database, String tableName, String[ * @return the dialects {@code UPDATE} statement. */ default String getUpdateStatement( - String database, String tableName, String[] fieldNames, String[] conditionFields) { + String database, + String tableName, + String[] fieldNames, + String[] conditionFields, + boolean isPrimaryKeyUpdated) { + + fieldNames = + Arrays.stream(fieldNames) + .filter( + fieldName -> + isPrimaryKeyUpdated + || !Arrays.asList(conditionFields) + .contains(fieldName)) + .toArray(String[]::new); + String setClause = Arrays.stream(fieldNames) .map(fieldName -> format("%s = :%s", quoteIdentifier(fieldName), fieldName)) @@ -200,4 +215,8 @@ default ResultSetMetaData getResultSetMetaData( PreparedStatement ps = conn.prepareStatement(jdbcSourceConfig.getQuery()); return ps.getMetaData(); } + + default String extractTableName(TablePath tablePath) { + return tablePath.getSchemaAndTableName(); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java index 128b8ae4be95..c71dc3f76a15 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/mysql/MysqlDialect.java @@ -17,6 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.mysql; +import org.apache.seatunnel.api.table.catalog.TablePath; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.JdbcRowConverter; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectTypeMapper; @@ -78,4 +79,9 @@ public PreparedStatement creatPreparedStatement( statement.setFetchSize(Integer.MIN_VALUE); return statement; } + + @Override + public String extractTableName(TablePath tablePath) { + return tablePath.getTableName(); + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlserverJdbcRowConverter.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlserverJdbcRowConverter.java index 1c22737b6577..717293e4f360 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlserverJdbcRowConverter.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/sqlserver/SqlserverJdbcRowConverter.java @@ -17,12 +17,100 @@ package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.sqlserver; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.converter.AbstractJdbcRowConverter; +import java.math.BigDecimal; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; + public class SqlserverJdbcRowConverter extends AbstractJdbcRowConverter { @Override public String converterName() { return "Sqlserver"; } + + public PreparedStatement toExternal( + SeaTunnelRowType rowType, SeaTunnelRow row, PreparedStatement statement) + throws SQLException { + for (int fieldIndex = 0; fieldIndex < rowType.getTotalFields(); fieldIndex++) { + SeaTunnelDataType seaTunnelDataType = rowType.getFieldType(fieldIndex); + int statementIndex = fieldIndex + 1; + Object fieldValue = row.getField(fieldIndex); + if (fieldValue == null && seaTunnelDataType.getSqlType() != SqlType.BYTES) { + statement.setObject(statementIndex, null); + continue; + } + + switch (seaTunnelDataType.getSqlType()) { + case STRING: + statement.setString(statementIndex, (String) row.getField(fieldIndex)); + break; + case BOOLEAN: + statement.setBoolean(statementIndex, (Boolean) row.getField(fieldIndex)); + break; + case TINYINT: + statement.setByte(statementIndex, (Byte) row.getField(fieldIndex)); + break; + case SMALLINT: + statement.setShort(statementIndex, (Short) row.getField(fieldIndex)); + break; + case INT: + statement.setInt(statementIndex, (Integer) row.getField(fieldIndex)); + break; + case BIGINT: + statement.setLong(statementIndex, (Long) row.getField(fieldIndex)); + break; + case FLOAT: + statement.setFloat(statementIndex, (Float) row.getField(fieldIndex)); + break; + case DOUBLE: + statement.setDouble(statementIndex, (Double) row.getField(fieldIndex)); + break; + case DECIMAL: + statement.setBigDecimal(statementIndex, (BigDecimal) row.getField(fieldIndex)); + break; + case DATE: + LocalDate localDate = (LocalDate) row.getField(fieldIndex); + statement.setDate(statementIndex, java.sql.Date.valueOf(localDate)); + break; + case TIME: + LocalTime localTime = (LocalTime) row.getField(fieldIndex); + statement.setTime(statementIndex, java.sql.Time.valueOf(localTime)); + break; + case TIMESTAMP: + LocalDateTime localDateTime = (LocalDateTime) row.getField(fieldIndex); + statement.setTimestamp( + statementIndex, java.sql.Timestamp.valueOf(localDateTime)); + break; + case BYTES: + if (row.getField(fieldIndex) == null) { + statement.setBytes(statementIndex, new byte[0]); + break; + } + statement.setBytes(statementIndex, (byte[]) row.getField(fieldIndex)); + break; + case NULL: + statement.setNull(statementIndex, java.sql.Types.NULL); + break; + case MAP: + case ARRAY: + case ROW: + default: + throw new JdbcConnectorException( + CommonErrorCode.UNSUPPORTED_DATA_TYPE, + "Unexpected value: " + seaTunnelDataType); + } + } + return statement; + } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java index eec473512b77..c23619b5aade 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSink.java @@ -34,10 +34,10 @@ import org.apache.seatunnel.api.table.catalog.CatalogOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.factory.CatalogFactory; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.tidb.TiDBCatalogFactory; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSinkConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.exception.JdbcConnectorException; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; @@ -58,6 +58,7 @@ import java.util.Optional; import static org.apache.seatunnel.api.common.SeaTunnelAPIErrorCode.HANDLE_SAVE_MODE_FAILED; +import static org.apache.seatunnel.api.table.factory.FactoryUtil.discoverFactory; @AutoService(SeaTunnelSink.class) public class JdbcSink @@ -189,28 +190,36 @@ public DataSaveMode getUserConfigSaveMode() { public void handleSaveMode(DataSaveMode saveMode) { if (catalogTable != null) { Map catalogOptions = config.get(CatalogOptions.CATALOG_OPTIONS); - if (catalogOptions != null - && TiDBCatalogFactory.IDENTIFIER.equalsIgnoreCase( - catalogOptions.get(CommonOptions.FACTORY_ID.key()))) { + if (catalogOptions != null) { + String factoryId = catalogOptions.get(CommonOptions.FACTORY_ID.key()); if (StringUtils.isBlank(jdbcSinkConfig.getDatabase())) { return; } - try (Catalog catalog = - new TiDBCatalogFactory() - .createCatalog( - TiDBCatalogFactory.IDENTIFIER, - ReadonlyConfig.fromMap(new HashMap<>(catalogOptions)))) { - catalog.open(); - TablePath tablePath = - TablePath.of(jdbcSinkConfig.getDatabase(), jdbcSinkConfig.getTable()); - if (!catalog.databaseExists(jdbcSinkConfig.getDatabase())) { - catalog.createDatabase(tablePath, true); + CatalogFactory catalogFactory = + discoverFactory( + Thread.currentThread().getContextClassLoader(), + CatalogFactory.class, + factoryId); + if (catalogFactory != null) { + try (Catalog catalog = + catalogFactory.createCatalog( + catalogFactory.factoryIdentifier(), + ReadonlyConfig.fromMap(new HashMap<>(catalogOptions)))) { + catalog.open(); + TablePath tablePath = + TablePath.of( + jdbcSinkConfig.getDatabase() + + "." + + jdbcSinkConfig.getTable()); + if (!catalog.databaseExists(jdbcSinkConfig.getDatabase())) { + catalog.createDatabase(tablePath, true); + } + if (!catalog.tableExists(tablePath)) { + catalog.createTable(tablePath, catalogTable, true); + } + } catch (Exception e) { + throw new JdbcConnectorException(HANDLE_SAVE_MODE_FAILED, e); } - if (!catalog.tableExists(tablePath)) { - catalog.createTable(tablePath, catalogTable, true); - } - } catch (Exception e) { - throw new JdbcConnectorException(HANDLE_SAVE_MODE_FAILED, e); } } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java index ef3f985432ac..a26628ff3a43 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/sink/JdbcSinkFactory.java @@ -20,18 +20,22 @@ import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.sink.DataSaveMode; +import org.apache.seatunnel.api.table.catalog.CatalogOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; import org.apache.seatunnel.api.table.catalog.ConstraintKey; import org.apache.seatunnel.api.table.catalog.PrimaryKey; +import org.apache.seatunnel.api.table.catalog.TableIdentifier; import org.apache.seatunnel.api.table.connector.TableSink; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableFactoryContext; import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.config.JdbcSinkConfig; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectLoader; import org.apache.commons.collections4.CollectionUtils; +import org.apache.commons.lang3.StringUtils; import com.google.auto.service.AutoService; @@ -72,10 +76,49 @@ public String factoryIdentifier() { public TableSink createSink(TableFactoryContext context) { ReadonlyConfig config = context.getOptions(); CatalogTable catalogTable = context.getCatalogTable(); + Map catalogOptions = config.get(CatalogOptions.CATALOG_OPTIONS); Optional optionalTable = config.getOptional(TABLE); if (!optionalTable.isPresent()) { + catalogOptions = catalogOptions == null ? new HashMap<>() : catalogOptions; + String prefix = catalogOptions.get(JdbcCatalogOptions.TABLE_PREFIX.key()); + String suffix = catalogOptions.get(JdbcCatalogOptions.TABLE_SUFFIX.key()); + if (StringUtils.isNotEmpty(prefix) || StringUtils.isNotEmpty(suffix)) { + TableIdentifier tableId = catalogTable.getTableId(); + String tableName = + StringUtils.isNotEmpty(prefix) + ? prefix + tableId.getTableName() + : tableId.getTableName(); + tableName = StringUtils.isNotEmpty(suffix) ? tableName + suffix : tableName; + TableIdentifier newTableId = + TableIdentifier.of( + tableId.getCatalogName(), + tableId.getDatabaseName(), + tableId.getSchemaName(), + tableName); + catalogTable = + CatalogTable.of( + newTableId, + catalogTable.getTableSchema(), + catalogTable.getOptions(), + catalogTable.getPartitionKeys(), + catalogTable.getCatalogName()); + } Map map = config.toMap(); - map.put(TABLE.key(), catalogTable.getTableId().getTableName()); + if (StringUtils.isNotBlank(catalogOptions.get(JdbcCatalogOptions.SCHEMA.key()))) { + map.put( + TABLE.key(), + catalogOptions.get(JdbcCatalogOptions.SCHEMA.key()) + + "." + + catalogTable.getTableId().getTableName()); + } else if (StringUtils.isNotBlank(catalogTable.getTableId().getSchemaName())) { + map.put( + TABLE.key(), + catalogTable.getTableId().getSchemaName() + + "." + + catalogTable.getTableId().getTableName()); + } else { + map.put(TABLE.key(), catalogTable.getTableId().getTableName()); + } PrimaryKey primaryKey = catalogTable.getTableSchema().getPrimaryKey(); if (primaryKey != null && !CollectionUtils.isEmpty(primaryKey.getColumnNames())) { @@ -104,13 +147,14 @@ public TableSink createSink(TableFactoryContext context) { JdbcDialectLoader.load( sinkConfig.getJdbcConnectionConfig().getUrl(), sinkConfig.getJdbcConnectionConfig().getCompatibleMode()); + CatalogTable finalCatalogTable = catalogTable; return () -> new JdbcSink( options, sinkConfig, dialect, DataSaveMode.KEEP_SCHEMA_AND_DATA, - catalogTable); + finalCatalogTable); } @Override diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java new file mode 100644 index 000000000000..daf87b3693a0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/mysql/MySqlCatalogTest.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver.SqlServerCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver.SqlServerURLParser; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.MethodOrderer; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; + +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +@Disabled("Please Test it in your local environment") +class MySqlCatalogTest { + + static JdbcUrlUtil.UrlInfo sqlParse = + SqlServerURLParser.parse("jdbc:sqlserver://127.0.0.1:1434;database=TestDB"); + static JdbcUrlUtil.UrlInfo MysqlUrlInfo = + JdbcUrlUtil.getUrlInfo("jdbc:mysql://127.0.0.1:33061/liuliTest?useSSL=false"); + static JdbcUrlUtil.UrlInfo pg = + JdbcUrlUtil.getUrlInfo("jdbc:postgresql://127.0.0.1:5432/liulitest"); + static TablePath tablePathSQL; + static TablePath tablePathMySql; + static TablePath tablePathPG; + static TablePath tablePathOracle; + private static String databaseName = "liuliTest"; + private static String schemaName = "dbo"; + private static String tableName = "AllDataTest"; + + static SqlServerCatalog sqlServerCatalog; + static MySqlCatalog mySqlCatalog; + static PostgresCatalog postgresCatalog; + + static CatalogTable postgresCatalogTable; + static CatalogTable mySqlCatalogTable; + static CatalogTable sqlServerCatalogTable; + + @Test + void listDatabases() {} + + @Test + void listTables() {} + + @Test + void getColumnsDefaultValue() {} + + @BeforeAll + static void before() { + tablePathSQL = TablePath.of(databaseName, "sqlserver_to_mysql"); + tablePathMySql = TablePath.of(databaseName, "mysql_to_mysql"); + tablePathPG = TablePath.of(databaseName, "pg_to_mysql"); + tablePathOracle = TablePath.of(databaseName, "oracle_to_mysql"); + sqlServerCatalog = new SqlServerCatalog("sqlserver", "sa", "root@123", sqlParse, null); + mySqlCatalog = new MySqlCatalog("mysql", "root", "root@123", MysqlUrlInfo); + postgresCatalog = new PostgresCatalog("postgres", "postgres", "postgres", pg, null); + mySqlCatalog.open(); + sqlServerCatalog.open(); + postgresCatalog.open(); + } + + @Test + @Order(1) + void getTable() { + postgresCatalogTable = + postgresCatalog.getTable( + TablePath.of("liulitest", "public", "pg_types_table_no_array")); + mySqlCatalogTable = mySqlCatalog.getTable(TablePath.of("liuliTest", "AllTypeCol")); + sqlServerCatalogTable = + sqlServerCatalog.getTable(TablePath.of("TestDB", "dbo", "AllDataTest")); + } + + @Test + @Order(2) + void createTableInternal() { + mySqlCatalog.createTable(tablePathMySql, mySqlCatalogTable, true); + mySqlCatalog.createTable(tablePathPG, postgresCatalogTable, true); + mySqlCatalog.createTable(tablePathSQL, sqlServerCatalogTable, true); + } + + @Disabled + // Manually dropping tables + @Test + void dropTableInternal() { + mySqlCatalog.dropTable(tablePathSQL, true); + mySqlCatalog.dropTable(tablePathMySql, true); + mySqlCatalog.dropTable(tablePathPG, true); + } + + @Test + void createDatabaseInternal() {} + + @Test + void dropDatabaseInternal() {} + + @AfterAll + static void after() { + sqlServerCatalog.close(); + mySqlCatalog.close(); + postgresCatalog.close(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java new file mode 100644 index 000000000000..6b8c49bc0abf --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/oracle/OracleCatalogTest.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MySqlCatalog; + +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +import java.util.List; + +@Disabled("Please Test it in your local environment") +class OracleCatalogTest { + @Test + void testCatalog() { + OracleCatalog catalog = + new OracleCatalog( + "oracle", + "test", + "oracle", + OracleURLParser.parse("jdbc:oracle:thin:@127.0.0.1:1521:xe"), + null); + + catalog.open(); + + MySqlCatalog mySqlCatalog = + new MySqlCatalog( + "mysql", + "root", + "root@123", + JdbcUrlUtil.getUrlInfo("jdbc:mysql://127.0.0.1:33062/mingdongtest")); + + mySqlCatalog.open(); + + CatalogTable table1 = + mySqlCatalog.getTable(TablePath.of("mingdongtest", "all_types_table_02")); + + List strings = catalog.listDatabases(); + System.out.println(strings); + + List strings1 = catalog.listTables("XE"); + + CatalogTable table = catalog.getTable(TablePath.of("XE", "TEST", "PG_TYPES_TABLE_CP1")); + + catalog.createTableInternal(new TablePath("XE", "TEST", "TEST003"), table); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java new file mode 100644 index 000000000000..badab864fc3f --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/psql/PostgresCatalogTest.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MySqlCatalog; + +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +@Disabled("Please Test it in your local environment") +class PostgresCatalogTest { + + @Test + void testCatalog() { + JdbcUrlUtil.UrlInfo urlInfo = + JdbcUrlUtil.getUrlInfo("jdbc:postgresql://127.0.0.1:5432/liulitest"); + PostgresCatalog catalog = + new PostgresCatalog("postgres", "postgres", "postgres", urlInfo, null); + + catalog.open(); + + MySqlCatalog mySqlCatalog = + new MySqlCatalog( + "mysql", + "root", + "root@123", + JdbcUrlUtil.getUrlInfo("jdbc:mysql://127.0.0.1:33062/mingdongtest")); + + mySqlCatalog.open(); + + CatalogTable table1 = + mySqlCatalog.getTable(TablePath.of("mingdongtest", "all_types_table_02")); + + CatalogTable table = + catalog.getTable(TablePath.of("st_test", "public", "all_types_table_02")); + System.out.println("find table: " + table); + + catalog.createTableInternal( + new TablePath("liulitest", "public", "all_types_table_02"), table); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/MysqlCreateTableSqlBuilderTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/MysqlCreateTableSqlBuilderTest.java index 3f84de199eb8..3de5c65bf8d7 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/MysqlCreateTableSqlBuilderTest.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sql/MysqlCreateTableSqlBuilderTest.java @@ -89,16 +89,17 @@ public void testBuild() { new ArrayList<>(), "User table"); - String createTableSql = MysqlCreateTableSqlBuilder.builder(tablePath, catalogTable).build(); + String createTableSql = + MysqlCreateTableSqlBuilder.builder(tablePath, catalogTable).build("mysql"); + // create table sql is change; The old unit tests are no longer applicable String expect = "CREATE TABLE IF NOT EXISTS test_table (\n" - + "\tid BIGINT (22) NOT NULL COMMENT 'id', \n" - + "\tname VARCHAR (128) NOT NULL COMMENT 'name', \n" - + "\tage INT NULL COMMENT 'age', \n" - + "\tcreateTime TIMESTAMP (3) NULL COMMENT 'createTime', \n" - + "\tlastUpdateTime TIMESTAMP (3) NULL COMMENT 'lastUpdateTime', \n" - + "\tPRIMARY KEY (`id`), \n" - + "\tKEY `name` (`name`)\n" + + "\tid null NOT NULL COMMENT 'id', \n" + + "\tname null NOT NULL COMMENT 'name', \n" + + "\tage null NULL COMMENT 'age', \n" + + "\tcreateTime null NULL COMMENT 'createTime', \n" + + "\tlastUpdateTime null NULL COMMENT 'lastUpdateTime', \n" + + "\tPRIMARY KEY (`id`)\n" + ") COMMENT = 'User table';"; CONSOLE.println(expect); Assertions.assertEquals(expect, createTableSql); diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java new file mode 100644 index 000000000000..5e457910f038 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerCatalogTest.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MySqlCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresCatalog; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.MethodOrderer; +import org.junit.jupiter.api.Order; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestMethodOrder; + +import java.util.List; + +@TestMethodOrder(MethodOrderer.OrderAnnotation.class) +@Disabled("Please Test it in your local environment") +class SqlServerCatalogTest { + + static JdbcUrlUtil.UrlInfo sqlParse = + SqlServerURLParser.parse("jdbc:sqlserver://127.0.0.1:1434;database=TestDB"); + static JdbcUrlUtil.UrlInfo MysqlUrlInfo = + JdbcUrlUtil.getUrlInfo("jdbc:mysql://127.0.0.1:33061/liuliTest?useSSL=false"); + static JdbcUrlUtil.UrlInfo pg = + JdbcUrlUtil.getUrlInfo("jdbc:postgresql://127.0.0.1:5432/liulitest"); + static TablePath tablePathSQL; + static TablePath tablePathMySql; + static TablePath tablePathPG; + static TablePath tablePathOracle; + private static String databaseName = "TestDB"; + private static String schemaName = "dbo"; + private static String tableName = "AllDataTest"; + + static SqlServerCatalog sqlServerCatalog; + static MySqlCatalog mySqlCatalog; + static PostgresCatalog postgresCatalog; + + static CatalogTable postgresCatalogTable; + static CatalogTable mySqlCatalogTable; + static CatalogTable sqlServerCatalogTable; + + @BeforeAll + static void before() { + tablePathSQL = TablePath.of(databaseName, schemaName, "sqlserver_to_sqlserver"); + tablePathMySql = TablePath.of(databaseName, schemaName, "mysql_to_sqlserver"); + tablePathPG = TablePath.of(databaseName, schemaName, "pg_to_sqlserver"); + tablePathOracle = TablePath.of(databaseName, schemaName, "oracle_to_sqlserver"); + sqlServerCatalog = new SqlServerCatalog("sqlserver", "sa", "root@123", sqlParse, null); + mySqlCatalog = new MySqlCatalog("mysql", "root", "root@123", MysqlUrlInfo); + postgresCatalog = new PostgresCatalog("postgres", "postgres", "postgres", pg, null); + mySqlCatalog.open(); + sqlServerCatalog.open(); + postgresCatalog.open(); + } + + @Test + void listDatabases() { + List list = sqlServerCatalog.listDatabases(); + } + + @Test + void listTables() { + List list = sqlServerCatalog.listTables(databaseName); + } + + @Test + void tableExists() { + + // boolean b = sqlServerCatalog.tableExists(tablePath); + } + + @Test + @Order(1) + void getTable() { + postgresCatalogTable = + postgresCatalog.getTable( + TablePath.of("liulitest", "public", "pg_types_table_no_array")); + mySqlCatalogTable = mySqlCatalog.getTable(TablePath.of("liuliTest", "AllTypeCol")); + sqlServerCatalogTable = + sqlServerCatalog.getTable(TablePath.of("TestDB", "dbo", "AllDataTest")); + } + + @Test + @Order(2) + void createTableInternal() { + sqlServerCatalog.createTable(tablePathMySql, mySqlCatalogTable, true); + sqlServerCatalog.createTable(tablePathPG, postgresCatalogTable, true); + sqlServerCatalog.createTable(tablePathSQL, sqlServerCatalogTable, true); + } + + @Disabled + // Manually dropping tables + @Test + void dropTableInternal() { + sqlServerCatalog.dropTable(tablePathSQL, true); + sqlServerCatalog.dropTable(tablePathMySql, true); + sqlServerCatalog.dropTable(tablePathPG, true); + } + + @Test + void createDatabaseInternal() {} + + @Test + void dropDatabaseInternal() {} + + @AfterAll + static void after() { + sqlServerCatalog.close(); + mySqlCatalog.close(); + postgresCatalog.close(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerURLParserTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerURLParserTest.java new file mode 100644 index 000000000000..a48b61ab0e56 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/catalog/sqlserver/SqlServerURLParserTest.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver; + +import org.apache.seatunnel.common.utils.JdbcUrlUtil; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +class SqlServerURLParserTest { + @Test + public void testParse() { + String url = + "jdbc:sqlserver://localhost:1433;databaseName=myDB;encrypt=true;trustServerCertificate=false;loginTimeout=30;"; + JdbcUrlUtil.UrlInfo urlInfo = SqlServerURLParser.parse(url); + assertEquals("localhost", urlInfo.getHost()); + assertEquals(1433, urlInfo.getPort()); + assertEquals(url, urlInfo.getOrigin()); + assertEquals( + "encrypt=true;trustServerCertificate=false;loginTimeout=30", urlInfo.getSuffix()); + assertEquals("myDB", urlInfo.getDefaultDatabase().get()); + assertEquals( + "jdbc:sqlserver://localhost:1433;encrypt=true;trustServerCertificate=false;loginTimeout=30", + urlInfo.getUrlWithoutDatabase()); + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf index 12846c6a0c21..614380ab3040 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf @@ -19,7 +19,7 @@ env { # You can set engine configuration here execution.parallelism = 1 job.mode = "STREAMING" - execution.checkpoint.interval = 5000 + checkpoint.interval = 5000 } source { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql.conf index e8d85aecc5cf..0adf2f7e64d0 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mysql-e2e/src/test/resources/mysqlcdc_to_mysql.conf @@ -22,7 +22,7 @@ env { # You can set engine configuration here execution.parallelism = 1 job.mode = "STREAMING" - execution.checkpoint.interval = 5000 + checkpoint.interval = 5000 } source { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/resources/sqlservercdc_to_console.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/resources/sqlservercdc_to_console.conf index c4ac06877b13..9d3f041ede14 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/resources/sqlservercdc_to_console.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-sqlserver-e2e/src/test/resources/sqlservercdc_to_console.conf @@ -51,8 +51,8 @@ sink { user = "sa" password = "Password!" generate_sink_sql = true - database = "" - table = "column_type_test.dbo.full_types_sink" + database = "column_type_test" + table = "dbo.full_types_sink" batch_size = 1 primary_keys = ["id"] } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml index 81ecdc298821..8628e2b80b63 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/pom.xml @@ -52,8 +52,30 @@ ${testcontainer.version} test
+ + org.testcontainers + oracle-xe + ${testcontainer.version} + test + + + org.testcontainers + mysql + ${testcontainer.version} + test + + + mysql + mysql-connector-java + test + + + com.oracle.database.jdbc + ojdbc8 + test + org.postgresql postgresql diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSinkCDCChangelogIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSinkCDCChangelogIT.java index 2a29c1cb5fe7..dd812efb12bc 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSinkCDCChangelogIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSinkCDCChangelogIT.java @@ -126,6 +126,16 @@ public void testSinkCDCChangelog(TestContainer container) Stream.>of(Arrays.asList(1L, "A_1", 100), Arrays.asList(3L, "C", 100)) .collect(Collectors.toSet()); Assertions.assertIterableEquals(expected, actual); + try (Connection connection = + DriverManager.getConnection( + postgreSQLContainer.getJdbcUrl(), + postgreSQLContainer.getUsername(), + postgreSQLContainer.getPassword())) { + try (Statement statement = connection.createStatement()) { + statement.execute("truncate table sink"); + log.info("testSinkCDCChangelog truncate table sink"); + } + } } private void initializeJdbcTable() { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_postgres_source_and_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_postgres_source_and_sink.conf index 1c7417f8a552..f3293f44e610 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_postgres_source_and_sink.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_postgres_source_and_sink.conf @@ -41,7 +41,7 @@ sink { password = test generate_sink_sql = true database = test - table = "public.pg_e2e_sink_table" + table = public.pg_e2e_sink_table primary_keys = ["gid"] } } \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_sink_cdc_changelog.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_sink_cdc_changelog.conf index 5a48476171e9..e0742a04f4cf 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_sink_cdc_changelog.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-3/src/test/resources/jdbc_sink_cdc_changelog.conf @@ -66,7 +66,7 @@ sink { password = test generate_sink_sql = true database = test - table = "public.sink" + table = public.sink primary_keys = ["pk_id"] } } \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/pom.xml new file mode 100644 index 000000000000..99bbff4fa23c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/pom.xml @@ -0,0 +1,96 @@ + + + + 4.0.0 + + org.apache.seatunnel + connector-jdbc-e2e + ${revision} + + + connector-jdbc-e2e-part-4 + SeaTunnel : E2E : Connector V2 : Jdbc : Part 4 + + + + org.apache.seatunnel + connector-jdbc-e2e-common + ${project.version} + test-jar + test + + + + + org.testcontainers + postgresql + ${testcontainer.version} + test + + + net.snowflake + snowflake-jdbc + test + + + org.testcontainers + mssqlserver + ${testcontainer.version} + test + + + org.testcontainers + oracle-xe + ${testcontainer.version} + test + + + org.testcontainers + mysql + ${testcontainer.version} + test + + + + + mysql + mysql-connector-java + test + + + com.oracle.database.jdbc + ojdbc8 + test + + + org.postgresql + postgresql + test + + + com.microsoft.sqlserver + mssql-jdbc + test + + + com.vertica.jdbc + vertica-jdbc + test + + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java new file mode 100644 index 000000000000..cdc6fe1992aa --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcMySqlCreateTableIT.java @@ -0,0 +1,471 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MySqlCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleURLParser; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver.SqlServerCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver.SqlServerURLParser; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.MSSQLServerContainer; +import org.testcontainers.containers.MySQLContainer; +import org.testcontainers.containers.OracleContainer; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.stream.Stream; + +@Slf4j +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "Currently SPARK and FLINK do not support cdc") +public class JdbcMySqlCreateTableIT extends TestSuiteBase implements TestResource { + private static final String SQLSERVER_IMAGE = "mcr.microsoft.com/mssql/server:2022-latest"; + private static final String SQLSERVER_CONTAINER_HOST = "sqlserver"; + private static final int SQLSERVER_CONTAINER_PORT = 14333; + private static final String DRIVER_CLASS = "com.microsoft.sqlserver.jdbc.SQLServerDriver"; + + private static final String PG_IMAGE = "postgis/postgis"; + private static final String PG_DRIVER_JAR = + "https://repo1.maven.org/maven2/org/postgresql/postgresql/42.3.3/postgresql-42.3.3.jar"; + private static final String PG_JDBC_JAR = + "https://repo1.maven.org/maven2/net/postgis/postgis-jdbc/2.5.1/postgis-jdbc-2.5.1.jar"; + private static final String PG_GEOMETRY_JAR = + "https://repo1.maven.org/maven2/net/postgis/postgis-geometry/2.5.1/postgis-geometry-2.5.1.jar"; + + private static final String MYSQL_IMAGE = "mysql:latest"; + private static final String MYSQL_CONTAINER_HOST = "mysql-e2e"; + private static final String MYSQL_DATABASE = "auto"; + + private static final String MYSQL_USERNAME = "root"; + private static final String PASSWORD = "Abc!@#135_seatunnel"; + private static final int MYSQL_PORT = 33061; + // private static final String MYSQL_URL = "jdbc:mysql://" + HOST + ":%s/%s?useSSL=false"; + + private static final String MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; + + private static final String ORACLE_IMAGE = "gvenzl/oracle-xe:21-slim-faststart"; + private static final String ORACLE_NETWORK_ALIASES = "e2e_oracleDb"; + private static final String ORACLE_DRIVER_CLASS = "oracle.jdbc.OracleDriver"; + private static final int ORACLE_PORT = 15211; + // private static final String ORACLE_URL = "jdbc:oracle:thin:@" + HOST + ":%s/%s"; + private static final String USERNAME = "testUser"; + private static final String DATABASE = "TESTUSER"; + + private PostgreSQLContainer POSTGRESQL_CONTAINER; + + private MSSQLServerContainer sqlserver_container; + private MySQLContainer mysql_container; + private OracleContainer oracle_container; + + private static final String mysqlCheck = + "SELECT EXISTS(SELECT 1 FROM information_schema.tables WHERE table_schema = 'auto' AND table_name = 'mysql_auto_create_mysql') AS table_exists"; + private static final String sqlserverCheck = + "IF EXISTS (\n" + + " SELECT 1\n" + + " FROM testauto.sys.tables t\n" + + " JOIN testauto.sys.schemas s ON t.schema_id = s.schema_id\n" + + " WHERE t.name = 'mysql_auto_create_sql' AND s.name = 'dbo'\n" + + ")\n" + + " SELECT 1 AS table_exists;\n" + + "ELSE\n" + + " SELECT 0 AS table_exists;"; + private static final String pgCheck = + "SELECT EXISTS(SELECT 1 FROM information_schema.tables WHERE table_schema = 'public' AND table_name = 'mysql_auto_create_pg') AS table_exists;\n"; + private static final String oracleCheck = + "SELECT CASE WHEN EXISTS(SELECT 1 FROM user_tables WHERE table_name = 'mysql_auto_create_oracle') THEN 1 ELSE 0 END AS table_exists FROM DUAL;\n"; + + String driverSqlServerUrl() { + return "https://repo1.maven.org/maven2/com/microsoft/sqlserver/mssql-jdbc/9.4.1.jre8/mssql-jdbc-9.4.1.jre8.jar"; + } + + private static final String CREATE_SQL_DATABASE = + "IF NOT EXISTS (\n" + + " SELECT name \n" + + " FROM sys.databases \n" + + " WHERE name = N'testauto'\n" + + ")\n" + + "CREATE DATABASE testauto;\n"; + + private static final String CREATE_TABLE_SQL = + "CREATE TABLE IF NOT EXISTS mysql_auto_create\n" + + "(\n " + + "`id` int(11) NOT NULL AUTO_INCREMENT,\n" + + " `f_binary` binary(64) DEFAULT NULL,\n" + + " `f_smallint` smallint(6) DEFAULT NULL,\n" + + " `f_smallint_unsigned` smallint(5) unsigned DEFAULT NULL,\n" + + " `f_mediumint` mediumint(9) DEFAULT NULL,\n" + + " `f_mediumint_unsigned` mediumint(8) unsigned DEFAULT NULL,\n" + + " `f_int` int(11) DEFAULT NULL,\n" + + " `f_int_unsigned` int(10) unsigned DEFAULT NULL,\n" + + " `f_integer` int(11) DEFAULT NULL,\n" + + " `f_integer_unsigned` int(10) unsigned DEFAULT NULL,\n" + + " `f_bigint` bigint(20) DEFAULT NULL,\n" + + " `f_bigint_unsigned` bigint(20) unsigned DEFAULT NULL,\n" + + " `f_numeric` decimal(10,0) DEFAULT NULL,\n" + + " `f_decimal` decimal(10,0) DEFAULT NULL,\n" + + " `f_float` float DEFAULT NULL,\n" + + " `f_double` double DEFAULT NULL,\n" + + " `f_double_precision` double DEFAULT NULL,\n" + + " `f_tinytext` tinytext COLLATE utf8mb4_unicode_ci,\n" + + " `f_varchar` varchar(100) COLLATE utf8mb4_unicode_ci DEFAULT NULL,\n" + + " `f_datetime` datetime DEFAULT NULL,\n" + + " `f_timestamp` timestamp NULL DEFAULT NULL,\n" + + " `f_bit1` bit(1) DEFAULT NULL,\n" + + " `f_bit64` bit(64) DEFAULT NULL,\n" + + " `f_char` char(1) COLLATE utf8mb4_unicode_ci DEFAULT NULL,\n" + + " `f_enum` enum('enum1','enum2','enum3') COLLATE utf8mb4_unicode_ci DEFAULT NULL,\n" + + " `f_real` double DEFAULT NULL,\n" + + " `f_tinyint` tinyint(4) DEFAULT NULL,\n" + + " `f_bigint8` bigint(8) DEFAULT NULL,\n" + + " `f_bigint1` bigint(1) DEFAULT NULL,\n" + + " `f_data` date DEFAULT NULL,\n" + + " PRIMARY KEY (`id`)\n" + + ");"; + + private String getInsertSql = + "INSERT INTO mysql_auto_create" + + "(id, f_binary, f_smallint, f_smallint_unsigned, f_mediumint, f_mediumint_unsigned, f_int, f_int_unsigned, f_integer, f_integer_unsigned, f_bigint, f_bigint_unsigned, f_numeric, f_decimal, f_float, f_double, f_double_precision, f_tinytext, f_varchar, f_datetime, f_timestamp, f_bit1, f_bit64, f_char, f_enum, f_real, f_tinyint, f_bigint8, f_bigint1, f_data)\n" + + "VALUES(575, 0x654458436C70336B7357000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000, 194, 549, 633, 835, 719, 253, 742, 265, 806, 736, 474, 254, 120.8, 476.42, 264.95, 'In other words, Navicat provides the ability for data in different databases and/or schemas to be kept up-to-date so that each repository contains the same information.', 'jF9X70ZqH4', '2011-10-20 23:10:08', '2017-09-10 19:33:51', 1, b'0001001101100000001010010100010111000010010110110101110011111100', 'u', 'enum2', 876.55, 25, 503, 1, '2011-03-06');\n"; + + @TestContainerExtension + private final ContainerExtendedFactory extendedSqlServerFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && curl -O " + + PG_DRIVER_JAR + + " && curl -O " + + PG_JDBC_JAR + + " && curl -O " + + PG_GEOMETRY_JAR + + " && curl -O " + + MYSQL_DRIVER_CLASS + + " && curl -O " + + ORACLE_DRIVER_CLASS + + " && curl -O " + + driverSqlserverUrl() + + " && curl -O " + + driverMySqlUrl() + + " && curl -O " + + driverOracleUrl()); + // Assertions.assertEquals(0, extraCommands.getExitCode()); + }; + + String driverMySqlUrl() { + return "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; + } + + String driverOracleUrl() { + return "https://repo1.maven.org/maven2/com/oracle/database/jdbc/ojdbc8/12.2.0.1/ojdbc8-12.2.0.1.jar"; + } + + String driverSqlserverUrl() { + return "https://repo1.maven.org/maven2/com/microsoft/sqlserver/mssql-jdbc/9.4.1.jre8/mssql-jdbc-9.4.1.jre8.jar"; + } + + void initContainer() throws ClassNotFoundException { + DockerImageName imageName = DockerImageName.parse(SQLSERVER_IMAGE); + sqlserver_container = + new MSSQLServerContainer<>(imageName) + .withNetwork(TestSuiteBase.NETWORK) + .withNetworkAliases(SQLSERVER_CONTAINER_HOST) + .withPassword(PASSWORD) + .acceptLicense() + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(SQLSERVER_IMAGE))); + + sqlserver_container.setPortBindings( + Lists.newArrayList(String.format("%s:%s", SQLSERVER_CONTAINER_PORT, 1433))); + + try { + Class.forName(sqlserver_container.getDriverClassName()); + } catch (ClassNotFoundException e) { + throw new SeaTunnelRuntimeException( + JdbcITErrorCode.DRIVER_NOT_FOUND, "Not found suitable driver for mssql", e); + } + + // ============= PG + POSTGRESQL_CONTAINER = + new PostgreSQLContainer<>( + DockerImageName.parse(PG_IMAGE) + .asCompatibleSubstituteFor("postgres")) + .withNetwork(TestSuiteBase.NETWORK) + .withNetworkAliases("postgresql") + .withDatabaseName("pg") + .withUsername(USERNAME) + .withPassword(PASSWORD) + .withCommand("postgres -c max_prepared_transactions=100") + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(PG_IMAGE))); + POSTGRESQL_CONTAINER.setPortBindings( + Lists.newArrayList(String.format("%s:%s", 54323, 5432))); + // Startables.deepStart(Stream.of(POSTGRESQL_CONTAINER)).join(); + log.info("PostgreSQL container started"); + Class.forName(POSTGRESQL_CONTAINER.getDriverClassName()); + + log.info("pg data initialization succeeded. Procedure"); + DockerImageName mysqlImageName = DockerImageName.parse(MYSQL_IMAGE); + mysql_container = + new MySQLContainer<>(mysqlImageName) + .withUsername(MYSQL_USERNAME) + .withPassword(PASSWORD) + .withDatabaseName(MYSQL_DATABASE) + .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_CONTAINER_HOST) + .withExposedPorts(MYSQL_PORT) + .waitingFor(Wait.forHealthcheck()) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(MYSQL_IMAGE))); + + mysql_container.setPortBindings( + Lists.newArrayList(String.format("%s:%s", MYSQL_PORT, 3306))); + DockerImageName oracleImageName = DockerImageName.parse(ORACLE_IMAGE); + oracle_container = + new OracleContainer(oracleImageName) + .withDatabaseName(DATABASE) + .withUsername(USERNAME) + .withPassword(PASSWORD) + .withNetwork(NETWORK) + .withNetworkAliases(ORACLE_NETWORK_ALIASES) + .withExposedPorts(ORACLE_PORT) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(ORACLE_IMAGE))); + oracle_container.withCommand( + "bash", + "-c", + "echo \"CREATE USER admin IDENTIFIED BY admin; GRANT DBA TO admin;\" | sqlplus / as sysdba"); + oracle_container.setPortBindings( + Lists.newArrayList(String.format("%s:%s", ORACLE_PORT, 1521))); + Startables.deepStart( + Stream.of( + POSTGRESQL_CONTAINER, + sqlserver_container, + mysql_container, + oracle_container)) + .join(); + } + + @Override + @BeforeAll + public void startUp() throws Exception { + initContainer(); + initializeSqlJdbcTable(); + initializeJdbcTable(); + } + + static JdbcUrlUtil.UrlInfo sqlParse = + SqlServerURLParser.parse("jdbc:sqlserver://localhost:14333;database=testauto"); + static JdbcUrlUtil.UrlInfo MysqlUrlInfo = + JdbcUrlUtil.getUrlInfo("jdbc:mysql://localhost:33061/auto?useSSL=false"); + static JdbcUrlUtil.UrlInfo pg = JdbcUrlUtil.getUrlInfo("jdbc:postgresql://localhost:54323/pg"); + static JdbcUrlUtil.UrlInfo oracle = + OracleURLParser.parse("jdbc:oracle:thin:@localhost:15211/TESTUSER"); + + @TestTemplate + public void testAutoCreateTable(TestContainer container) + throws IOException, InterruptedException { + TablePath tablePathMySql = TablePath.of("auto", "mysql_auto_create"); + TablePath tablePathMySql_Mysql = TablePath.of("auto", "mysql_auto_create_mysql"); + TablePath tablePathSQL = TablePath.of("testauto", "dbo", "mysql_auto_create_sql"); + TablePath tablePathPG = TablePath.of("pg", "public", "mysql_auto_create_pg"); + TablePath tablePathOracle = TablePath.of("TESTUSER", "mysql_auto_create_oracle"); + + SqlServerCatalog sqlServerCatalog = + new SqlServerCatalog("sqlserver", "sa", PASSWORD, sqlParse, "dbo"); + MySqlCatalog mySqlCatalog = new MySqlCatalog("mysql", "root", PASSWORD, MysqlUrlInfo); + PostgresCatalog postgresCatalog = + new PostgresCatalog("postgres", "testUser", PASSWORD, pg, "public"); + OracleCatalog oracleCatalog = + new OracleCatalog("oracle", "admin", "admin", oracle, "TESTUSER"); + mySqlCatalog.open(); + sqlServerCatalog.open(); + postgresCatalog.open(); + // oracleCatalog.open(); + + CatalogTable mysqlTable = mySqlCatalog.getTable(tablePathMySql); + + sqlServerCatalog.createTable(tablePathSQL, mysqlTable, true); + postgresCatalog.createTable(tablePathPG, mysqlTable, true); + // oracleCatalog.createTable(tablePathOracle, mysqlTable, true); + mySqlCatalog.createTable(tablePathMySql_Mysql, mysqlTable, true); + + Assertions.assertTrue(checkMysql(mysqlCheck)); + // Assertions.assertTrue(checkOracle(oracleCheck)); + Assertions.assertTrue(checkSqlServer(sqlserverCheck)); + Assertions.assertTrue(checkPG(pgCheck)); + + // delete table + log.info("delete table"); + mySqlCatalog.dropTable(tablePathMySql_Mysql, true); + sqlServerCatalog.dropTable(tablePathSQL, true); + postgresCatalog.dropTable(tablePathPG, true); + // oracleCatalog.dropTable(tablePathOracle, true); + mySqlCatalog.dropTable(tablePathMySql, true); + + sqlServerCatalog.close(); + mySqlCatalog.close(); + postgresCatalog.close(); + // delete table + } + + @Override + public void tearDown() throws Exception { + + sqlserver_container.close(); + mysql_container.close(); + oracle_container.close(); + POSTGRESQL_CONTAINER.close(); + } + + private Connection getJdbcSqlServerConnection() throws SQLException { + return DriverManager.getConnection( + sqlserver_container.getJdbcUrl(), + sqlserver_container.getUsername(), + sqlserver_container.getPassword()); + } + + private Connection getJdbcMySqlConnection() throws SQLException { + return DriverManager.getConnection( + mysql_container.getJdbcUrl(), + mysql_container.getUsername(), + mysql_container.getPassword()); + } + + private Connection getJdbcPgConnection() throws SQLException { + return DriverManager.getConnection( + POSTGRESQL_CONTAINER.getJdbcUrl(), + POSTGRESQL_CONTAINER.getUsername(), + POSTGRESQL_CONTAINER.getPassword()); + } + + private Connection getJdbcOracleConnection() throws SQLException { + return DriverManager.getConnection( + oracle_container.getJdbcUrl(), + oracle_container.getUsername(), + oracle_container.getPassword()); + } + + private void initializeSqlJdbcTable() { + try (Connection connection = getJdbcSqlServerConnection()) { + Statement statement = connection.createStatement(); + statement.execute(CREATE_SQL_DATABASE); + // statement.executeBatch(); + } catch (SQLException e) { + throw new RuntimeException("Initializing PostgreSql table failed!", e); + } + } + + private void initializeJdbcTable() { + try (Connection connection = getJdbcMySqlConnection()) { + Statement statement = connection.createStatement(); + statement.execute(CREATE_TABLE_SQL); + statement.execute(getInsertSql); + + // statement.executeBatch(); + } catch (SQLException e) { + throw new RuntimeException("Initializing PostgreSql table failed!", e); + } + } + + private boolean checkMysql(String sql) { + try (Connection connection = getJdbcMySqlConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + boolean tableExists = false; + if (resultSet.next()) { + tableExists = resultSet.getBoolean(1); + } + return tableExists; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private boolean checkPG(String sql) { + try (Connection connection = getJdbcPgConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + boolean tableExists = false; + if (resultSet.next()) { + tableExists = resultSet.getBoolean(1); + } + return tableExists; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private boolean checkSqlServer(String sql) { + try (Connection connection = getJdbcSqlServerConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + boolean tableExists = false; + if (resultSet.next()) { + tableExists = resultSet.getInt(1) == 1; + } + return tableExists; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private boolean checkOracle(String sql) { + try (Connection connection = getJdbcOracleConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + boolean tableExists = false; + if (resultSet.next()) { + tableExists = resultSet.getInt(1) == 1; + } + return tableExists; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java new file mode 100644 index 000000000000..35a2338b2605 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/connector-jdbc-e2e-part-4/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/JdbcSqlServerCreateTableIT.java @@ -0,0 +1,482 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc; + +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.mysql.MySqlCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.oracle.OracleURLParser; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver.SqlServerCatalog; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.sqlserver.SqlServerURLParser; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; +import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; +import org.testcontainers.containers.MSSQLServerContainer; +import org.testcontainers.containers.MySQLContainer; +import org.testcontainers.containers.OracleContainer; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.DockerImageName; +import org.testcontainers.utility.DockerLoggerFactory; + +import com.google.common.collect.Lists; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.stream.Stream; + +@Slf4j +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "Currently SPARK and FLINK do not support cdc") +public class JdbcSqlServerCreateTableIT extends TestSuiteBase implements TestResource { + + private static final String SQLSERVER_IMAGE = "mcr.microsoft.com/mssql/server:2022-latest"; + private static final String SQLSERVER_CONTAINER_HOST = "sqlserver-e2e"; + private static final int SQLSERVER_CONTAINER_PORT = 1433; + private static final String SQLSERVER_URL = + "jdbc:sqlserver://" + AbstractJdbcIT.HOST + ":%s;encrypt=false;"; + private static final String DRIVER_CLASS = "com.microsoft.sqlserver.jdbc.SQLServerDriver"; + + private static final String CREATE_DATABASE = + "IF NOT EXISTS (\n" + + " SELECT name \n" + + " FROM sys.databases \n" + + " WHERE name = N'testauto'\n" + + ")\n" + + "CREATE DATABASE testauto;\n"; + + private static final String CREATE_TABLE_SQL = + "IF NOT EXISTS (SELECT * FROM testauto.sys.tables WHERE name = 'sqlserver_auto_create' AND schema_id = SCHEMA_ID('dbo'))\n" + + "BEGIN\n" + + "CREATE TABLE testauto.dbo.sqlserver_auto_create (\n" + + " c1 bigint NOT NULL,\n" + + " c2 bit NULL,\n" + + " c3 decimal(18) NULL,\n" + + " c4 decimal(18,2) NULL,\n" + + " c5 real NULL,\n" + + " c6 float(53) NULL,\n" + + " c7 int NULL,\n" + + " c8 money NULL,\n" + + " c9 numeric(18) NULL,\n" + + " c10 numeric(18,2) NULL,\n" + + " c11 real NULL,\n" + + " c12 smallint NULL,\n" + + " c13 smallmoney NULL,\n" + + " c14 tinyint NULL,\n" + + " c15 char(10) NULL,\n" + + " c16 varchar(50) NULL,\n" + + " c17 varchar(max) NULL,\n" + + " c18 text NULL,\n" + + " c19 nchar(10) NULL,\n" + + " c20 nvarchar(50) NULL,\n" + + " c21 nvarchar(max) NULL,\n" + + " c22 ntext NULL,\n" + + " c25 varbinary(max) NULL,\n" + + " c26 image NULL,\n" + + " c27 datetime NULL,\n" + + " c28 datetime2(7) NULL,\n" + + " c29 datetimeoffset(7) NULL,\n" + + " c30 smalldatetime NULL,\n" + + " c31 date NULL,\n" + + " PRIMARY KEY CLUSTERED (c1)\n" + + ") \n" + + "END"; + + private String username; + + private String password; + + private String getInsertSql = + "INSERT INTO testauto.dbo.sqlserver_auto_create\n" + + "(c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18, c19, c20, c21, c22, c25, c26, c27, c28, c29, c30, c31)\n" + + "VALUES(8, 1, 714, 876.63, 368.74686, 61.59519333775628, 97, 7.1403, 497, 727.56, 303.78827, 654, 620.8399, 181, N'qEVAoi6KLU', N'1Y7QDYF6me', N'Navicat allows you to transfer data from one database and/or schema to another with detailed analytical process. Instead of wondering when your next vacation is, maybe you should set up a life you don’t need to escape from. I will greet this day with love in my heart. HTTP Tunneling is a method for connecting to a server that uses the same protocol (http://) and the same port (port 80) as a web server does. Export Wizard allows you to export data from tables, collections, views, or query results to any available formats. Always keep your eyes open. Keep watching. Because whatever you see can inspire you. After logged in the Navicat Cloud feature, the Navigation pane will be divided into Navicat Cloud and My Connections sections. Navicat Cloud could not connect and access your databases. By which it means, it could only store your connection settings, queries, model files, and virtual group; your database passwords and data (e.g. tables, views, etc) will not be stored to Navicat Cloud. Always keep your eyes open. Keep watching. Because whatever you see can inspire you. With its well-designed Graphical User Interface(GUI), Navicat lets you quickly and easily create, organize, access and share information in a secure and easy way. Anyone who has ever made anything of importance was disciplined. After logged in the Navicat Cloud feature, the Navigation pane will be divided into Navicat Cloud and My Connections sections. If you wait, all that happens is you get older. Navicat Data Modeler enables you to build high-quality conceptual, logical and physical data models for a wide variety of audiences. Navicat Monitor requires a repository to store alerts and metrics for historical analysis. There is no way to happiness. Happiness is the way. To connect to a database or schema, simply double-click it in the pane. Anyone who has never made a mistake has never tried anything new. If your Internet Service Provider (ISP) does not provide direct access to its server, Secure Tunneling Protocol (SSH) / HTTP is another solution. Navicat 15 has added support for the system-wide dark mode. You will succeed because most people are lazy. Success consists of going from failure to failure without loss of enthusiasm. SSH serves to prevent such vulnerabilities and allows you to access a remote server''s shell without compromising security. Navicat provides a wide range advanced features, such as compelling code editing capabilities, smart code-completion, SQL formatting, and more. Navicat provides powerful tools for working with queries: Query Editor for editing the query text directly, and Query Builder, Find Builder or Aggregate Builder for building queries visually. The Synchronize to Database function will give you a full picture of all database differences. If the plan doesn’t work, change the plan, but never the goal. You can select any connections, objects or projects, and then select the corresponding buttons on the Information Pane. The Main Window consists of several toolbars and panes for you to work on connections, database objects and advanced tools. Actually it is just in an idea when feel oneself can achieve and cannot achieve. The Main Window consists of several toolbars and panes for you to work on connections, database objects and advanced tools. After logged in the Navicat Cloud feature, the Navigation pane will be divided into Navicat Cloud and My Connections sections. Anyone who has never made a mistake has never tried anything new. Navicat Monitor is a safe, simple and agentless remote server monitoring tool that is packed with powerful features to make your monitoring effective as possible. The Main Window consists of several toolbars and panes for you to work on connections, database objects and advanced tools. Navicat provides a wide range advanced features, such as compelling code editing capabilities, smart code-completion, SQL formatting, and more. Champions keep playing until they get it right. If it scares you, it might be a good thing to try. It can also manage cloud databases such as Amazon Redshift, Amazon RDS, Alibaba Cloud. Features in Navicat are sophisticated enough to provide professional developers for all their specific needs, yet easy to learn for users who are new to database server. To connect to a database or schema, simply double-click it in the pane. A query is used to extract data from the database in a readable format according to the user''s request. To successfully establish a new connection to local/remote server - no matter via SSL or SSH, set the database login information in the General tab. SQL Editor allows you to create and edit SQL text, prepare and execute selected queries. Navicat is a multi-connections Database Administration tool allowing you to connect to MySQL, Oracle, PostgreSQL, SQLite, SQL Server, MariaDB and/or MongoDB databases, making database administration to multiple kinds of database so easy. Secure Sockets Layer(SSL) is a protocol for transmitting private documents via the Internet. I may not have gone where I intended to go, but I think I have ended up where I needed to be. Navicat Cloud provides a cloud service for synchronizing connections, queries, model files and virtual group information from Navicat, other Navicat family members, different machines and different platforms. To connect to a database or schema, simply double-click it in the pane. With its well-designed Graphical User Interface(GUI), Navicat lets you quickly and easily create, organize, access and share information in a secure and easy way. I may not have gone where I intended to go, but I think I have ended up where I needed to be. Anyone who has ever made anything of importance was disciplined. Actually it is just in an idea when feel oneself can achieve and cannot achieve. Instead of wondering when your next vacation is, maybe you should set up a life you don’t need to escape from. It wasn’t raining when Noah built the ark. You must be the change you wish to see in the world. SQL Editor allows you to create and edit SQL text, prepare and execute selected queries. Navicat provides a wide range advanced features, such as compelling code editing capabilities, smart code-completion, SQL formatting, and more. To start working with your server in Navicat, you should first establish a connection or several connections using the Connection window. SSH serves to prevent such vulnerabilities and allows you to access a remote server''s shell without compromising security. In the Objects tab, you can use the List List, Detail Detail and ER Diagram ER Diagram buttons to change the object view. Genius is an infinite capacity for taking pains. Typically, it is employed as an encrypted version of Telnet. Secure Sockets Layer(SSL) is a protocol for transmitting private documents via the Internet. You cannot save people, you can just love them. You cannot save people, you can just love them. Navicat provides a wide range advanced features, such as compelling code editing capabilities, smart code-completion, SQL formatting, and more. To connect to a database or schema, simply double-click it in the pane. Navicat provides a wide range advanced features, such as compelling code editing capabilities, smart code-completion, SQL formatting, and more. Navicat Monitor requires a repository to store alerts and metrics for historical analysis. How we spend our days is, of course, how we spend our lives. Instead of wondering when your next vacation is, maybe you should set up a life you don’t need to escape from. To start working with your server in Navicat, you should first establish a connection or several connections using the Connection window. Always keep your eyes open. Keep watching. Because whatever you see can inspire you. Navicat Data Modeler enables you to build high-quality conceptual, logical and physical data models for a wide variety of audiences. Navicat Cloud could not connect and access your databases. By which it means, it could only store your connection settings, queries, model files, and virtual group; your database passwords and data (e.g. tables, views, etc) will not be stored to Navicat Cloud. I may not have gone where I intended to go, but I think I have ended up where I needed to be. The reason why a great man is great is that he resolves to be a great man. Export Wizard allows you to export data from tables, collections, views, or query results to any available formats. Navicat 15 has added support for the system-wide dark mode. Actually it is just in an idea when feel oneself can achieve and cannot achieve. SSH serves to prevent such vulnerabilities and allows you to access a remote server''s shell without compromising security. Difficult circumstances serve as a textbook of life for people. Flexible settings enable you to set up a custom key for comparison and synchronization. It collects process metrics such as CPU load, RAM usage, and a variety of other resources over SSH/SNMP. It wasn’t raining when Noah built the ark. SQL Editor allows you to create and edit SQL text, prepare and execute selected queries. You can select any connections, objects or projects, and then select the corresponding buttons on the Information Pane.', N'Actually it is just in an idea when feel oneself can achieve and cannot achieve. A man is not old until regrets take the place of dreams. With its well-designed Graphical User Interface(GUI), Navicat lets you quickly and easily create, organize, access and share information in a secure and easy way.', N'j8OKNCrsFb', N'KTLmoNjIiI', N'All the Navicat Cloud objects are located under different projects. You can share the project to other Navicat Cloud accounts for collaboration. Navicat Data Modeler is a powerful and cost-effective database design tool which helps you build high-quality conceptual, logical and physical data models. After logged in the Navicat Cloud feature, the Navigation pane will be divided into Navicat Cloud and My Connections sections. Navicat Cloud provides a cloud service for synchronizing connections, queries, model files and virtual group information from Navicat, other Navicat family members, different machines and different platforms. Secure Sockets Layer(SSL) is a protocol for transmitting private documents via the Internet. To successfully establish a new connection to local/remote server - no matter via SSL, SSH or HTTP, set the database login information in the General tab. Champions keep playing until they get it right. It is used while your ISPs do not allow direct connections, but allows establishing HTTP connections. With its well-designed Graphical User Interface(GUI), Navicat lets you quickly and easily create, organize, access and share information in a secure and easy way. Navicat allows you to transfer data from one database and/or schema to another with detailed analytical process. You must be the change you wish to see in the world. Navicat provides a wide range advanced features, such as compelling code editing capabilities, smart code-completion, SQL formatting, and more. Anyone who has never made a mistake has never tried anything new. Navicat allows you to transfer data from one database and/or schema to another with detailed analytical process. I may not have gone where I intended to go, but I think I have ended up where I needed to be. Typically, it is employed as an encrypted version of Telnet. Secure SHell (SSH) is a program to log in into another computer over a network, execute commands on a remote server, and move files from one machine to another. Success consists of going from failure to failure without loss of enthusiasm. Sometimes you win, sometimes you learn. Navicat 15 has added support for the system-wide dark mode. It provides strong authentication and secure encrypted communications between two hosts, known as SSH Port Forwarding (Tunneling), over an insecure network.', N'To connect to a database or schema, simply double-click it in the pane. If you wait, all that happens is you get older. Always keep your eyes open. Keep watching. Because whatever you see can inspire you. Import Wizard allows you to import data to tables/collections from CSV, TXT, XML, DBF and more. Success consists of going from failure to failure without loss of enthusiasm. A query is used to extract data from the database in a readable format according to the user''s request. Anyone who has never made a mistake has never tried anything new. To successfully establish a new connection to local/remote server - no matter via SSL or SSH, set the database login information in the General tab. SQL Editor allows you to create and edit SQL text, prepare and execute selected queries. Navicat Monitor is a safe, simple and agentless remote server monitoring tool that is packed with powerful features to make your monitoring effective as possible. I will greet this day with love in my heart. How we spend our days is, of course, how we spend our lives. You can select any connections, objects or projects, and then select the corresponding buttons on the Information Pane. Remember that failure is an event, not a person. The Information Pane shows the detailed object information, project activities, the DDL of database objects, object dependencies, membership of users/roles and preview. Navicat authorizes you to make connection to remote servers running on different platforms (i.e. Windows, macOS, Linux and UNIX), and supports PAM and GSSAPI authentication. Secure Sockets Layer(SSL) is a protocol for transmitting private documents via the Internet. The Information Pane shows the detailed object information, project activities, the DDL of database objects, object dependencies, membership of users/roles and preview. You can select any connections, objects or projects, and then select the corresponding buttons on the Information Pane. The On Startup feature allows you to control what tabs appear when you launch Navicat. The first step is as good as half over. Always keep your eyes open. Keep watching. Because whatever you see can inspire you. Champions keep playing until they get it right. If the Show objects under schema in navigation pane option is checked at the Preferences window, all database objects are also displayed in the pane. To successfully establish a new connection to local/remote server - no matter via SSL, SSH or HTTP, set the database login information in the General tab. It provides strong authentication and secure encrypted communications between two hosts, known as SSH Port Forwarding (Tunneling), over an insecure network. Navicat is a multi-connections Database Administration tool allowing you to connect to MySQL, Oracle, PostgreSQL, SQLite, SQL Server, MariaDB and/or MongoDB databases, making database administration to multiple kinds of database so easy. It wasn’t raining when Noah built the ark. A comfort zone is a beautiful place, but nothing ever grows there. Navicat Cloud provides a cloud service for synchronizing connections, queries, model files and virtual group information from Navicat, other Navicat family members, different machines and different platforms. The past has no power over the present moment. Creativity is intelligence having fun. Navicat authorizes you to make connection to remote servers running on different platforms (i.e. Windows, macOS, Linux and UNIX), and supports PAM and GSSAPI authentication. HTTP Tunneling is a method for connecting to a server that uses the same protocol (http://) and the same port (port 80) as a web server does. Difficult circumstances serve as a textbook of life for people. A comfort zone is a beautiful place, but nothing ever grows there. I may not have gone where I intended to go, but I think I have ended up where I needed to be. It wasn’t raining when Noah built the ark. Navicat Cloud could not connect and access your databases. By which it means, it could only store your connection settings, queries, model files, and virtual group; your database passwords and data (e.g. tables, views, etc) will not be stored to Navicat Cloud. What you get by achieving your goals is not as important as what you become by achieving your goals. Difficult circumstances serve as a textbook of life for people. There is no way to happiness. Happiness is the way. Genius is an infinite capacity for taking pains. If the plan doesn’t work, change the plan, but never the goal. Genius is an infinite capacity for taking pains.', 0xFFD8FFE000104A46494600010100000100010000FFDB004300080606070605080707070909080A0C140D0C0B0B0C1912130F141D1A1F1E1D1A1C1C20242E2720222C231C1C2837292C30313434341F27393D38323C2E333432FFDB0043010909090C0B0C180D0D1832211C213232323232323232323232323232323232323232323232323232323232323232323232323232323232323232323232323232FFC00011080140014003012200021101031101FFC4001C0001010003000301000000000000000000000705060801020304FFC400441000010302020605070A0309010000000000010203040506110712213141B23651617172153542748191B113142223326282A1C1D152C2F016172433535493A2D2E1FFC4001A010100030101010000000000000000000000030405020601FFC400311101000201020306040602030000000000000102030411051231213233517181133441D114156191B1C152A12442F0FFDA000C03010002110311003F00A8000CB42000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000031376C4D66B1ECB857C5149FE922EB3FB3E8A66B976EE34FAED2ED0C6E54A1B64F3A6796B4AF48D3BD32D6FD09F1E97365EDA55DD715EDD214604827D2E5D9CEFA8B7D1313AA4D77FC150F9B34B57C45FA7456F54EC63D3F98B1F966A3CBFDA5FC36458C130A4D2FB55CD6D65A1513D27C3367EE6AA7EA6DB67C7161BD3DB14158914EECB286A1351CABD49C157B115487268F3E38DED5476C37AF586C4002B2300000035CC478D2DD862AA1A7AD86AA47CACD76AC2C6AA226796DCDC8754C76C96E5AC6F2FB5ACDA7686C60D0FF00BDAB0FFB4B97FC6CFF00D99FC398B2DB89D93AD124CC7C2A88E8E6444764BB953255D84B7D366A579AD5DA1D5B15EB1BCC33A0020700000000003E35752CA3A39EAA4472B218DD2391BBD511335CBDC693FDED587FDA5CBFE367FEC971E0C997B691BBAAD2D6EEC37C06A763D20DA6FF00758EDD4B4F5AC9A4472A3A56311BB133E0E5EA36C39C98AF8E796F1B496ACD676900070E40000000000000000000000FCD5F5F4D6CA09AB6AE548A085BACF72FF5BD57622088999DA0EAF171B951DAA89F575D3B21819BDCE5DFD889C57B1091626D255C2E6E7D35A95F45499AA7CA22E52BD3B57D1F67BCC1E29C515789AE4E9A473994AC5CA0833D8C4EB5EB72F153027A0D270FAE388B64ED9FE1A1874F15EDB7579739CF72B9CAAAE55CD55576AA9E0D830EE0EBAE247A3E9A248A95172754CBB189DDC5CBDDEDC8A45B345963A4622D73E7AE932DBACE58D9EC46AE7F9A96336B70E19DAD3DBE5092F9E94EC945C1D110612C3D4EC4632CB42A89FEA428F5F7BB353DA4C2D87E46EABACB6F44FBB4ED6FC10A9F9B63FF001945F8BAF939D416DB968C30FD6B55695B350C9B72589EAE6AAF6A3B3F722A133C4983AE786A4D6A8624D48E5C99531A7D155EA5EA5FEB696F06BB0E69DAB3B4FEA9699E97EC86430B6906E1637B29EB1CFACB7A6CD472E6F8D3EEAAFC176756459ADD71A4BAD0C7594533658244CD1C9C3B17A97B0E68366C198AE6C35736A48E73ADF33B29E3DF97DF4ED4FCD3675655F5BA0AE489BE38DADFCA3CD822D1CD5EABD83D63919344C9637A3E37B51CD735734545DCA87B1E7D9E122D2EF9E6DFEAEBCCA574916977CF36FF00575E652F70DF988F74FA6F1213B32787EF75187EF305C29F6EA2E52333D8F62EF6FF005C723180F496AC5A26B3D25A531131B4BA6682BA9EE541056D2BF5E09988F63BB3F73F411FD19E29F27D7791AAE4CA9AA5D9C2E72FD893ABB9DF1CBAD4B01E5755A79C1926B3D3E8CACB8E71DB600057460000C7DF7A3D72F5597914E6E3A46FBD1EB97AACBC8A7371B9C23B965ED27496DBA35E9C51F824E452E842F46BD38A3F049C8A5D0ABC57C78F4FBA2D577FD80019AAC00000000000000000000120D286237565C9B65A77AFCDE9575A6C9763E45E1F853F355EA2A379B8B2D166ACB83F25482257A22F15E09ED5C90E6F9A692A2792695EAF92472BDEE5DEAAAB9AA9ABC2F045AF3927E9FCAD6971EF3CD3F47A1BB603C17E5F9D6BEBDAE6DBA17648DDCB3BBA93B138AFB138E5A9DB6825BA5CE9A861FF00327912345EACD77FB379D1D6FA082D96F828A99BAB0C0C46353F55ED5DE5DE23AA9C34E5A7594FA8CB348DA3ACBED1451C10B2186364713111AD631A888D44DC8889B90F700F3ACE00000F8D552C15B4B25354C4D96191BAAF639334543EC044EDDB039FF18E1A7E19BD3A06E6EA49915F4EF5FE1E2D5ED4FD9789AF17AC7D644BCE16A8D4667514A8B3C59266AB926D4F6A67B3AF2221476CAFB8AAA51515454E4B92FC8C4AECBBF243D368B53F1716F69ED8EAD3C3979E9BCAAFA2DBF3ABAD32DAA77AACB47B62555DAB1AF0F62FE4A886FE4A702E11C4768C41057D452B29E9B55CC952495359CD54E0899EDCF25DB96E2AC626BA2919A671CEF13E4A59E2BCFBD4245A5DF3CDBFD5D7994AE922D2EF9E6DFEAEBCCA77C37E623DDF74DE242767DA6A59E9E385F2C6AD64ECF948D57739B9AA669ED453E25629F0CB71268B6DAD89A9F3EA763DF4EEEB5D7766DEE5F8E46F67CF187966DD26765FC99229B6E9422AA2A2A2AA2A6E542EB80F142621B324750FCEBE95119367BDE9C1FEDE3DBEC214F63A37B98F6AB5CD5C95AA992A2F5192C3F7BA8C3F7982E14FB751729199EC7B177B7FAE3911EB34D19F1ED1D63A39CD8FE257F574703F3D05753DCA820ADA57EBC13311EC7767EE7E83CBCC4C4ED2CBE80000C7DF7A3D72F5597914E6E3A46FBD1EB97AACBC8A7371B9C23B965ED27496DBA35E9C51F824E452E842F46BD38A3F049C8A5D0ABC57C78F4FBA2D577FD80019AAC00000000000000000000D0F4AD5CB4F8661A46B9116A67447275B5A99FC7548D14AD2FCDAD5B6A83F82391FEF544FE526A7A5E1D4E5D3C7EAD3D346D8E1BE68AADA95588E6AD7A66DA48736F63DDB13F2D62CA4EB4454C8CB35C2AB2DB254246BF85A8BFCE514C7E237E6D44FE9D8A7A8B6F924001490000000000A88A992ED43C22235A88888889B111381E4000000245A5DF3CDBFD5D7994AE922D2EF9E6DFEAEBCCA5EE1BF311EE9F4DE242765F700F41ED9E0773B88117DC03D07B6781DCEE3478B7831EBFD4AC6AFB91EAD2749D85BE6D51E5DA48FEA6672254B53D17F07772F1EDEF27074DD5D2C35D492D2D4C692432B558F6AF14539F3135826C397A96865CDD1FDA8645F4D8BB97BF82F6A0E1BAAE7AFC2B758FE1F74D979A3967AC365D1AE29F265C7C91572654954EFAA739764727ECED89DF9769643978B9E00C53FDA0B47C854C99DC29511B2AAEF91BC1FFA2F6F7A10713D2ED3F1ABEFF747AAC5FF00786DC0031D4D8FBEF47AE5EAB2F229CDC748DF7A3D72F5597914E6E37384772CBDA4E92DB746BD38A3F049C8A5D085E8D7A7147E093914BA1578AF8F1E9F745AAEFF00B00033558000000000000000000011DD2DBF3C4B46CCB751B573EF7BFF00634028BA5D8952F36F9783A9D5BEE77FF49D1EA7433FF1EAD4C1E1C2D7A2C6A3707AAA7A552F55F7221BB1A2E8A25D7C293B38C756F4F7B5ABFA9BD1E7F59E3DFD59F9BC4900056460000000000000000122D2EF9E6DFEAEBCCA574916977CF36FF575E652F70DF988F74FA6F1213B2FB807A0F6CF03B9DC408BEE01E83DB3C0EE771A3C5BC18F5FEA56357DC8F56C86B58DB0CB712595CD89A9F3EA7CDF4EEEB5E2DEE5F8E46CA0C3C792D8ED17AF5851ADA6B3BC397DEC746F731ED56B9AB92B5532545EA32162BCD4586EF05C29B6BA35C9CC55C91ED5DED5FEB7E4BC0DDB49D85BE6D51E5DA48FEA6672254B53D17F07772F1EDEF2707A9C5929A8C5CDF49EAD5A5A3257774BDBAE14D75B7C15D48FD78266EB357E28BDA8B9A2F71FA88DE8D714F932E3E48AB932A4AA77D539CBB2393F676C4EFCBB4B21E6F55A79C1926BF4FA3372E39C76D98FBEF47AE5EAB2F229CDC748DF7A3D72F5597914E6E35384772CB5A4E92DB746BD38A3F049C8A5D085E8D7A7147E093914BA1578AF8F1E9F745AAEFF00B000335580000000000000000000135D2F522BA82D9589BA395F12FE24454E452505F71EDBBCA583AB98D4CE485A93B3667F676AFE599023D170CBF360E5F2968E96DBD36F254B443588B1DD289576A2B256A7BD17E0D29E41B47D744B5E2FA557B91B154A2D3BD57EF6597FD91A5E4CCE278F973CCF9AB6A6BB64DFCC001415C000000003D5AF63F3D4735D92AB5725CF254E07A54D445474B2D4CEE46C51315EF72F0444CD4E709EE9572DD6A2E31CD2C13CD23A45746F5454D65CF2CD0B9A4D1CEA37EDDB64D8B0CE4DDD280845AF1E628A79A1822AE75566E463639D88FD655D889ADF6BF32EACD7F936FCA6AEBE49ADABBB3E391C6A74B7D3CC734C76BE65C538FABD8916977CF36FF575E652BA48B4BBE79B7FABAF3292F0DF988F775A6F1213B2FB807A0F6CF03B9DC408BEE01E83DB3C0EE771A3C5BC18F5FEA56357DC8F56C80030141F1ABA586BA925A5A98D248656AB1ED5E28A73E626B04D872F52D0CB9BA3FB50C8BE9B1772F7F05ED43A24D6B1B6196E24B2B9B1353E7D4F9BE9DDD6BC5BDCBF1C8BDA0D57C1C9B5BBB3FF00B74F832F25B69E9281973C018A7FB4168F90A9933B852A236555DF23783FF45EDEF421AF63A37B98F6AB5CD5C95AA992A2F5190B15E6A2C37782E14DB5D1AE4E62AE48F6AEF6AFF5BF25E06D6B34D19F1ED1D63A2EE6C7F12BB7D5D017DE8F5CBD565E4539B8E88ACB8535D7075657523F5E09A8A57357F0AE68BDA8B9A2F71CEE54E131315BC4F9A1D246D12DB746BD38A3F049C8A5D085E8D7A7147E093914BA1538AF8F1E9F745AAEFF00B0003355800000000000000000001E1CD6BDAAD72239AA992A2A668A873A624B43AC5882AE8151518C7E712AF162ED6AFBBF3453A30D174958656ED6B4B9D2C7AD5746DFA4889B5F16F54F66D5F797F876A23165E5B74958D364E5B6D3F5465AE731C8E6AAA391734545DA8A5FB06E248F11D8D92B9EDF9E4288CA9626F4770765D4B967EF4E0400C958EF95B87EE4CADA27A23D1355EC77D97B7A950D8D6E97F114DA3AC745CCD8BE257F5747835FC398C2D789226A412A4557966FA6917E92777F1276A7B723603CD5E96A5B96D1B4B32D59ACED2000E5F000D4716E3BA2C3F1494D4CE654DCB2C92245CDB1AF5BD7F4DFDDBCEF1E2BE5B72D2379755ACDA76862B4A3889B4B6E6D969E4FF115393A6CBD18D3877AAA7B917AC909F6ABABA8AFAB96AEAA574B3CAED67BDDBD54F9318E91ED631AAE7397246A266AABD47A8D3608C18E29FBB4F1638C75D9B6E8E2CEEB9E2A867735160A2FAF7AAFF17A09DF9EDF617335AC1187530ED8238E56E5593E52D42F145E0DF626CEFCCD94C0D767F8D9A663A47642867C9CF7ECE8122D2EF9E6DFEAEBCCA574916977CF36FF00575E653AE1BF311EEFBA6F1213B2FB807A0F6CF03B9DC408BEE01E83DB3C0EE771A3C5BC18F5FEA56357DC8F56C80030140000125D27616F9B547976923FA999C8952D4F45FC1DDCBC7B7BC9C1D37574B0D7524B4B531A490CAD563DABC514E7CC4D609B0E5EA5A1973747F6A1917D362EE5EFE0BDA86FF0DD573D7E15BAC7F0D0D365E68E59EB0CB612C51E4DB7DCAD156FFF0009554F2FC92AAEC8E4D45FC9DB13BF2ED35100D0AE3AD6D368FAAC456226663EADB746BD38A3F049C8A5D085E8D7A7147E093914BA185C57C78F4FBA86ABBFEC000CD560000000000000000000000004871EE047DBE496EF6A8D5D48E557CF0B536C2BC5C9F77E1DDBA787509A1E25D19D0DD1CFAAB53D94554BB56354FAA7AF727D9F66CEC36747C4A2239337EFF75CC3A9DA396E8DB5CE6391CD554722E68A8BB514DAAD7A45C456C8D235A9655C6D4C91B54DD754FC48A8E5F6A98DBAE15BDD955CB5B6F99B1B76FCB3135E3CBAF593627B72530C6ACD7167AF6ED685B98ADE3CD4C8B4C13237EBACB1BDDD6CA856A7E6D513E97E7731529ECD1C6FE0B2542BD3DC8D4266083F2FD36FBF2FFB947F87C7E4DA2EBA41C457563A3755A52C4EDECA56EA7FDB6BBF33570676CF83AF97BD57D2D0BDB03B2FAF9BE8332EB455DFECCC9E23160AFD2B0936AD23C9822B1A3FC0AFA47C77ABB44AD9D36D3D3BD36B3EFB93AFA9386FDFBB3385F47D6FB03995552A9595EDDA9239B93235FBA9D7DABB7B8DC0C7D6711E789C78BA79A9E6D473472D000192A8122D2EF9E6DFEAEBCCA574916977CF36FF00575E652F70DF988F74FA6F1213B2FB807A0F6CF03B9DC408BEE01E83DB3C0EE771A3C5BC18F5FEA56357DC8F56C80030140000035AC6D865B892CAE6C4D4F9F53E6FA7775AF16F72FC723650778F25B1DA2F5EB0FB5B4D677872FBD8E8DEE63DAAD7357256AA64A8BD47828FA4EC2DF36A8F2ED247F5333912A5A9E8BF83BB978F6F79383D560CD5CD8E2F56B63BC5EBCD0DB746BD38A3F049C8A5D085E8D7A7147E093914BA189C57C78F4FBA8EABBFEC000CD5600000000000000000000000000000C7D5D86D15EED6AAB6524CFF00E27C2D577BF2CCC803EC5A6BDB1244CC746B52600C2F2FDAB4B13C32BDBF071E19A3EC2D1BB59B6A6AAFDE9A4727B95C6CC097F119BFCE7F7977F12FE72C751582CF6E735F476CA48646EC491B126B27E2DE644022B5A6D3BCCEEE26667A8003E00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000FFD9, 0xFFD8FFE000104A46494600010100000100010000FFDB004300080606070605080707070909080A0C140D0C0B0B0C1912130F141D1A1F1E1D1A1C1C20242E2720222C231C1C2837292C30313434341F27393D38323C2E333432FFDB0043010909090C0B0C180D0D1832211C213232323232323232323232323232323232323232323232323232323232323232323232323232323232323232323232323232FFC00011080140014003012200021101031101FFC4001C0001000301010101010000000000000000000506070804030201FFC40049100100010302020507070906040700000000010203040511063107213641615171727481B1B2121314152223911735547393A1B3C1D224528292D1F03242626325335355A2A4E2FFC400190101000301010000000000000000000000000304050201FFC400251101000201030304030100000000000000000102030411321231332141718113226151FFDA000C03010002110311003F00D24064AD000000000000000000000000000000000000000000000000000000000000000004F5472DC0148C8E94347C4C9B98F7F0353B77AD5534574556A8DE2639C7FC6F9FE56342FD1352FD9D1FD697F064FF00117E6C7FEAF622B40E20C1E23D3E733066B8A69AA68AEDDC888AE89F18899F3A551CC4C4ED29226263780078F400000011FAD6B389A0E99733F36AAA2D5131114D11BD554CF2888EF9553F2B1A17E89A97ECE8FEB775C57B46F58716C95ACED32BD8A4E374A1A46664DBC7C7C0D4EE5EB954534514DAA37999FF001AED1D71CB6797A5A9CA1ED6F5B719079751CEB5A669D919D7A9AEAB562DCDCAA28889AA623C9BA9DF958D0BF44D4BF6747F5BDAE3BDFD6B0F2D92B5E52BD8AA689D20695AF6A96F4FC5C7CDA2F5C8AA62ABB45314F546FDD54F916B796A5AB3B5A1ED6D168DE001CBA000000054758E91348D1755BFA764E3E6D77ACCC45555BA289A677889EADEA8F2BC5F958D0BF44D4BF6747F5A58C192637884739A913B4CAF63E183976F3F031B32D45516F22D537688AA3AE22A8898DFC7ADF745D9D800F40000000000000019A749DC2FF396FEBEC3B7F6E888A72A9A639C728AFD9CA7C36F232C74E5CB745EB55DAB94C576EBA669AA9AA378989E712C138C786EBE1BD6ABB34C4CE25EDEBC7AE7FBBDF4CF8C72FC27BDA3A4CDBC744A86A716D3D70FEF06F11D5C39AE517ABAA7E877B6A322988DFECF755B79639FE3E56F74574DCA29AE8AA2AA2A8DE9AA99DE263CB0E626B1D187134E4E3CE859773EF6CD3F2B1A667AEAA3BE9F67778798D5E1DE3AE0D2E5DA7A25A300CE5F0000145E9238A3EABD3BEAAC4B9B666553F6EAA67AEDDBEFF6CF5C79B7F0778E937B4561C5EF14AF54A91C7DC4DF5F6B33631AE7CAC0C599A6D4C4F557577D5FCA3C23C552167E08E18AF88B5889BD44FD031E62ABF57F7BC94479FDDBF835FF005C54FE432FF6C97FECAE1D19F0ACE359FAF732DFDEDDA76C5A663AE9A279D7EDE51E1BF95A3BF94D34D14C534C4534C46D1111B4443FAC8C992725BAA5A98E914AF4C21B8B7B23AB7AB57EE73D3A178B7B23AB7AB57EE73D2F68B8CA9EAF942D9D1BF6DF0FD0B9F04B7261BD1BF6DF0FD0B9F04B7243ACF27D25D270FB00545A00000060DC7FDB8D4FD2A3F874AB4B2F1FF6E353F4A8FE1D2AD36B170AFC43232739F974570D765B48F52B3F0425117C35D96D23D4ACFC109463DF94B56BC6001CBA00000000000000010BC53C3F6B88F45BB8756D4DFA7EDD8B93FF002D71CBD93CA5343DADA6B3BC3C988B46D2E65C8C7BB8993731EFDB9B77AD553457455CE2639C3F787977F0332CE5E35C9B77ACD715D154774C34CE93B85FE72DFD7D876FEDD1114E5534C738E515FB394F86DE4658D9C592325376564A4E3B6CE89E1CD72CF10E8B633ED6D4D557D9BB6E277F915C738FE71E13095615C0DC4F3C3DACC517EB9FA064CC517A3BA89EEAFD9DFE133E0DD62626378EB865E7C5F8EDFC6861C9F92BFD004299E1D6355C7D134ABFA8654FDDDAA778A639D73DD4C78CCB9EB54D4B2357D4F233F2AADEF5FAFE54EDCA23944478446D1EC5A3A43E279D6B56FA0E357BE0E255311B72B97394D5E68E51EDF2A98D4D2E1E8AF54F7966EA32F5DB68ED0F4E9F8191A9EA1630B168F977EF5514D31FCE7C239BA0787B44B1C3FA359C0B1B4CD31F2AED7B6DF395CF3ABFDF7442AFD1BF0ACE9983F5B6651B65E4D3F754CC75DBB7FEB3CFCDB78AF8ADAACDD73D31DA1634D8BA63AA7BC802A2D21B8B7B23AB7AB57EE73D3A178B7B23AB7AB57EE73D34745C654357CA16CE8DFB6F87E85CF825B930DE8DFB6F87E85CF825B921D6793E92E9387D802A2D000000306E3FEDC6A7E951FC3A55A5978FFB71A9FA547F0E9569B58B857E2191939CFCBA2B86BB2DA47A959F821288BE1AECB691EA567E084A31EFCA5AB5E3000E5D0000000000000000003F372DD17AD576AE5315DBAE99A6AA6A8DE26279C4B01E2EE1DB9C37ADDCC6DA6716E6F731EB9EFA37E533E58E53EC9EF74020B8B7876DF126897317AA326DFDE63D73DD5EDCA7C2794FE3DC9F4F97F1DBD7B4A0CF8BAEBE9DDCFCD8BA35E269D4B4E9D272AE6F958B4C7CD4CCF5D76FFD69EA8F34C78B20BD66E63DFB962F51345DB754D15D1573A6A89DA625E9D2B52BFA46A98F9F8D3B5DB15FCA8F18E531E698DE3DAD2CD8E3253651C59271DB7749299D21F134E8BA47D0B1AE6D9B99134C4C4F5DBB7CA6AF3F747B67B960B3AEE0DEE1E8D6FE76230FE666ECCF7C6DCE3CFBC4C6DE560BAEEB17F5DD632350BFD5372AFB34F7514C754447B1434D87AAFBDBB42E6A32F4D768F7472E1C01C2BF5F6A7F4BCAA2274FC5AA26B8AA3AAE55CE29F37299F0EAEF57748D2B235AD52C6062D3BDCBB56DBF7531DF54F84475BA0B47D2B1F44D2AC69F8B1F776A9DA6A9E75CF7D53E332B5A9CDD15E98EF2ADA7C5D73BCF687B8065B480010DC5BD91D5BD5ABF739E9D0BC5BD91D5BD5ABF739E9A3A2E32A1ABE50B6746FDB7C3F42E7C12DC986F46FDB7C3F42E7C12DC90EB3C9F49749C3EC0151680000018371FF6E353F4A8FE1D2AD2CBC7FDB8D4FD2A3F874AB4DAC5C2BF10C8C9CE7E5D15C35D96D23D4ACFC109445F0D765B48F52B3F042518F7E52D5AF180072E800000000000000000000198F49DC2FBC7D7F876FAE36A72A9A63F0AFDD13ECF165EE9BBD66DE458B966F514D76AE5334574551BC55131B4C4B02E2EE1DAF86F5CB98D1BD58D73EF31EB9EFA26794F8C729FC7BDA3A4CDD51D12A1A9C5B4F5C3C14EB39B4E87568F177FB1D57A2F4D3DFF002B6E5E6EFDBCAF00BDF471C2D1AAE7CEAB9746F898B5C7CDD33CAE5CE7F84754FB63C566F6AE3ACDA55EB59BDA2AB8F47DC2BF5169BF4DCAA36CFCAA6266263AED51CE29F3F7CFB23B972063DEF37B754B56958AC6D000E5D00021B8B7B23AB7AB57EE73D3A178B7B23AB7AB57EE73D34745C654357CA16CE8DFB6F87E85CF825B930DE8DFB6F87E85CF825B921D6793E92E9387D802A2D000000306E3FEDC6A7E951FC3A55A5978FF00B71A9FA547F0E9569B58B857E2191939CFCBA2B86BB2DA47A959F821288BE1AECB691EA567E084A31EFCA5AB5E3000E5D0000000000000000000000C9FA5CFCE5A6FEA6BF7C35864FD2E7E72D37F535FBE1634BE5841A9F1CB396CFD16764ABF5AAFDD4B186CFD16764ABF5AAFDD4AE6B3C6A9A5F22EE032DA400000087E2CA66AE12D5A23F45B93FB9CF2E91D62CFD2744CFB1FFAB8D728FC6998737347453FACC286AE3F685A3A3BAE69E39D3E37DA2A8B913FB3A9BB39F783B263178C34BB957544DF8A3FCDF67F9BA0916B63F789FE25D24FE9200A6B4000000C0F8EEBF97C6DA9CFFDCA63F0A2985752BC4D91F4BE28D52F44EF1564DC889F08AA623F74229B78E36A4431EF3BDA65D17C3B4CD1C31A4D3546D31876627FC909379F06CFD1B4FC6B1B6DF376A9A36F34443D0C5B4EF332D7AC6D1000F1E8000000000000000000000C9FA5CFCE5A6FEA6BF7C35864FD2E7E72D37F535FBE1634BE5841A9F1CB396CFD16764ABF5AAFDD4B186CFD16764ABF5AAFDD4AE6B3C6A9A5F22EE032DA40000004C44C6D3D70E6DD5B0A74DD5F330A63FF0022F556E3CD13D5FB9D24C77A52D1E70F5EB7A9514CFCD6653B55311D515D31113F8C6DFBD6F477DAF35FF557555DEB13FE28F62F578F916EFDB9DAE5BAA2BA67C9313BC3A474ECEB5A9E9D8F9D6277B57EDC574F86F1CBCF1C9CD4BD700F1AD1A255F566A354C605CAB7A2E6DBFCD553CF7FFA67F77E2B3AAC537AEF1DE1069B2452DB4F696C83F16AEDBBD6A9BB6AE5372DD51BD35513BC4C784BF6CB6880008FD7353B7A3E89979F72A88F99B7334EFDF572A63DB3B43DB76EDBB36AABB76E536EDD31BD55573B44478CB1CE3FE32A35DBB4E9DA7D754E0DAABE5575F2F9EABBBFC31FEFB92E1C5392DB7B22CB92295DFDD489999999999999E73291E1FC29D438874FC588DFE72FD1157A3BEF3FBB746B43E8AB45AAFEA77F58B94FDD635336AD4CC73B95475EDE6A67FF00943572DFA2932CDC75EABC435B018AD7000000000000000000000000193F4B9F9CB4DFD4D7EF86B0C9FA5CFCE5A6FEA6BF7C2C697CB08353E39672D9FA2CEC957EB55FBA9630D9FA2CEC957EB55FBA95CD678D534BE45DC065B4800000046EBDA2E3F106917B4FC999A62BEBA2B88DE68AA39551FEFCA921EC4CC4EF0F262263697376ADA4E668BA8DCC1CEB5345DA394F7571DD5533DF12F13A275DE1DD3B88B0FE8F9D6779A77F9BBB4F5576E7C27F972651AE746FACE99555730E9FAC31E3AE26D47DB8F3D1CE7D9BB4F16A6B78DADE92CECBA7B56778F5841E91C4BABE873FF87E6DCB76FBED4ED5513FE19EAF6C75ADF8BD2DE7D14ED97A663DE9F2DAAEAB7EFF0094CF6ED9BB8F76AB57ADD76EE533B554574CC4C4F8C4BF09AD8B1DFD6611D72DEBE912D3E7A5F9DBAB43EBF5BFFF000F0E5F4B5A9DC8DB134FC5B3E3726AB93FC99F0E234D8A3D9D4EA324FBA5756E24D635B9DB50CEBB768EEB71B5347F96368F6A29F4B18F7F2AF53671ECDCBD76AFF868B74CD554FB2175D07A32D5350AE9BBA9FF0061C6E7F26769BB579A3BBDBD7E0EED6A638F5F47115BE49F4F556B40D033788B52A70F0E9EAE772ECC7D9B74F967FD3BDBEE93A663E8DA5D8C0C5A76B5669DB79E754F7D53E333D6FE693A3E0E89854E26058A6D5B8EB99FF9AB9F2D53DF2F73373E79CB3B4766861C318E379EE00AE9C000000000000000000000000000000000000000000001F0C9C2C5CDA3E4656359BF47F76EDB8AA3F7A22F70570DDF9DEBD231E27FE889A3DD309E1D45AD1DA5CCD6B3DE158FC9DF0AFF00ED7FFD8BBFD4F558E0BE1BC7AFE551A4634CFF00DC89AE3F0AB74E8F672DE7DE5E7E3A47B43E38D898D876A2D62E3DAB16E39516A88A63F087D81C3B0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000007FFD9, '2006-02-27 05:15:03.000', '2019-08-14 17:36:43.000', N'2003-05-14 08:07:42 +00:00', '1900-06-19 00:00:00.000', '2005-05-29');\n"; + + private static final String PG_IMAGE = "postgis/postgis"; + private static final String PG_DRIVER_JAR = + "https://repo1.maven.org/maven2/org/postgresql/postgresql/42.3.3/postgresql-42.3.3.jar"; + private static final String PG_JDBC_JAR = + "https://repo1.maven.org/maven2/net/postgis/postgis-jdbc/2.5.1/postgis-jdbc-2.5.1.jar"; + private static final String PG_GEOMETRY_JAR = + "https://repo1.maven.org/maven2/net/postgis/postgis-geometry/2.5.1/postgis-geometry-2.5.1.jar"; + + private static final String MYSQL_IMAGE = "mysql:latest"; + private static final String MYSQL_CONTAINER_HOST = "mysql-e2e"; + private static final String MYSQL_DATABASE = "auto"; + + private static final String MYSQL_USERNAME = "root"; + private static final String MYSQL_PASSWORD = "Abc!@#135_seatunnel"; + private static final int MYSQL_PORT = 3306; + // private static final String MYSQL_URL = "jdbc:mysql://" + HOST + ":%s/%s?useSSL=false"; + + private static final String MYSQL_DRIVER_CLASS = "com.mysql.cj.jdbc.Driver"; + + private static final String ORACLE_IMAGE = "gvenzl/oracle-xe:21-slim-faststart"; + private static final String ORACLE_NETWORK_ALIASES = "e2e_oracleDb"; + private static final String ORACLE_DRIVER_CLASS = "oracle.jdbc.OracleDriver"; + private static final int ORACLE_PORT = 1521; + // private static final String ORACLE_URL = "jdbc:oracle:thin:@" + HOST + ":%s/%s"; + private static final String USERNAME = "testUser"; + private static final String PASSWORD = "Abc!@#135_seatunnel"; + private static final String DATABASE = "TESTUSER"; + private static final String SOURCE_TABLE = "E2E_TABLE_SOURCE"; + private static final String SINK_TABLE = "E2E_TABLE_SINK"; + + private PostgreSQLContainer POSTGRESQL_CONTAINER; + + private MSSQLServerContainer sqlserver_container; + private MySQLContainer mysql_container; + private OracleContainer oracle_container; + + private static final String mysqlCheck = + "SELECT EXISTS(SELECT 1 FROM information_schema.tables WHERE table_schema = 'auto' AND table_name = 'sqlserver_auto_create_mysql') AS table_exists"; + private static final String sqlserverCheck = + "IF EXISTS (\n" + + " SELECT 1\n" + + " FROM testauto.sys.tables t\n" + + " JOIN testauto.sys.schemas s ON t.schema_id = s.schema_id\n" + + " WHERE t.name = 'sqlserver_auto_create_sql' AND s.name = 'dbo'\n" + + ")\n" + + " SELECT 1 AS table_exists;\n" + + "ELSE\n" + + " SELECT 0 AS table_exists;"; + private static final String pgCheck = + "SELECT EXISTS(SELECT 1 FROM information_schema.tables WHERE table_schema = 'public' AND table_name = 'sqlserver_auto_create_pg') AS table_exists;\n"; + private static final String oracleCheck = + "SELECT CASE WHEN EXISTS(SELECT 1 FROM user_tables WHERE table_name = 'sqlserver_auto_create_oracle') THEN 1 ELSE 0 END AS table_exists FROM DUAL;\n"; + + String driverMySqlUrl() { + return "https://repo1.maven.org/maven2/com/mysql/mysql-connector-j/8.0.32/mysql-connector-j-8.0.32.jar"; + } + + String driverOracleUrl() { + return "https://repo1.maven.org/maven2/com/oracle/database/jdbc/ojdbc8/12.2.0.1/ojdbc8-12.2.0.1.jar"; + } + + String driverSqlserverUrl() { + return "https://repo1.maven.org/maven2/com/microsoft/sqlserver/mssql-jdbc/9.4.1.jre8/mssql-jdbc-9.4.1.jre8.jar"; + } + + static JdbcUrlUtil.UrlInfo sqlParse = + SqlServerURLParser.parse("jdbc:sqlserver://localhost:1433;database=testauto"); + static JdbcUrlUtil.UrlInfo MysqlUrlInfo = + JdbcUrlUtil.getUrlInfo("jdbc:mysql://localhost:3306/auto?useSSL=false"); + static JdbcUrlUtil.UrlInfo pg = JdbcUrlUtil.getUrlInfo("jdbc:postgresql://localhost:5432/pg"); + static JdbcUrlUtil.UrlInfo oracle = + OracleURLParser.parse("jdbc:oracle:thin:@localhost:1521/TESTUSER"); + + @TestContainerExtension + private final ContainerExtendedFactory extendedSqlServerFactory = + container -> { + Container.ExecResult extraCommands = + container.execInContainer( + "bash", + "-c", + "mkdir -p /tmp/seatunnel/plugins/Jdbc/lib && cd /tmp/seatunnel/plugins/Jdbc/lib && curl -O " + + PG_DRIVER_JAR + + " && curl -O " + + PG_JDBC_JAR + + " && curl -O " + + PG_GEOMETRY_JAR + + " && curl -O " + + MYSQL_DRIVER_CLASS + + " && curl -O " + + ORACLE_DRIVER_CLASS + + " && curl -O " + + driverSqlserverUrl() + + " && curl -O " + + driverMySqlUrl() + + " && curl -O " + + driverOracleUrl()); + // Assertions.assertEquals(0, extraCommands.getExitCode()); + }; + + void initContainer() throws ClassNotFoundException { + DockerImageName imageName = DockerImageName.parse(SQLSERVER_IMAGE); + sqlserver_container = + new MSSQLServerContainer<>(imageName) + .withNetwork(TestSuiteBase.NETWORK) + .withNetworkAliases(SQLSERVER_CONTAINER_HOST) + .withPassword(PASSWORD) + .acceptLicense() + .withLogConsumer( + new Slf4jLogConsumer( + DockerLoggerFactory.getLogger(SQLSERVER_IMAGE))); + + sqlserver_container.setPortBindings( + Lists.newArrayList( + String.format( + "%s:%s", SQLSERVER_CONTAINER_PORT, SQLSERVER_CONTAINER_PORT))); + + try { + Class.forName(sqlserver_container.getDriverClassName()); + } catch (ClassNotFoundException e) { + throw new SeaTunnelRuntimeException( + JdbcITErrorCode.DRIVER_NOT_FOUND, "Not found suitable driver for mssql", e); + } + + username = sqlserver_container.getUsername(); + password = sqlserver_container.getPassword(); + // ============= PG + POSTGRESQL_CONTAINER = + new PostgreSQLContainer<>( + DockerImageName.parse(PG_IMAGE) + .asCompatibleSubstituteFor("postgres")) + .withNetwork(TestSuiteBase.NETWORK) + .withNetworkAliases("postgre-e2e") + .withDatabaseName("pg") + .withUsername(USERNAME) + .withPassword(PASSWORD) + .withCommand("postgres -c max_prepared_transactions=100") + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(PG_IMAGE))); + POSTGRESQL_CONTAINER.setPortBindings( + Lists.newArrayList(String.format("%s:%s", 5432, 5432))); + + log.info("PostgreSQL container started"); + Class.forName(POSTGRESQL_CONTAINER.getDriverClassName()); + + log.info("pg data initialization succeeded. Procedure"); + DockerImageName mysqlImageName = DockerImageName.parse(MYSQL_IMAGE); + mysql_container = + new MySQLContainer<>(mysqlImageName) + .withUsername(MYSQL_USERNAME) + .withPassword(MYSQL_PASSWORD) + .withDatabaseName(MYSQL_DATABASE) + .withNetwork(NETWORK) + .withNetworkAliases(MYSQL_CONTAINER_HOST) + .withExposedPorts(MYSQL_PORT) + .waitingFor(Wait.forHealthcheck()) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(MYSQL_IMAGE))); + + mysql_container.setPortBindings( + Lists.newArrayList(String.format("%s:%s", MYSQL_PORT, MYSQL_PORT))); + + DockerImageName oracleImageName = DockerImageName.parse(ORACLE_IMAGE); + oracle_container = + new OracleContainer(oracleImageName) + .withDatabaseName(DATABASE) + .withUsername(USERNAME) + .withPassword(PASSWORD) + .withNetwork(NETWORK) + .withNetworkAliases(ORACLE_NETWORK_ALIASES) + .withExposedPorts(ORACLE_PORT) + .withLogConsumer( + new Slf4jLogConsumer(DockerLoggerFactory.getLogger(ORACLE_IMAGE))); + oracle_container.withCommand( + "bash", + "-c", + "echo \"CREATE USER admin IDENTIFIED BY admin; GRANT DBA TO admin;\" | sqlplus / as sysdba"); + oracle_container.setPortBindings( + Lists.newArrayList(String.format("%s:%s", ORACLE_PORT, ORACLE_PORT))); + Startables.deepStart( + Stream.of( + POSTGRESQL_CONTAINER, + sqlserver_container, + mysql_container, + oracle_container)) + .join(); + + log.info(" container is up "); + } + + @Override + @BeforeAll + public void startUp() throws Exception { + initContainer(); + + initializeJdbcTable(); + } + + @TestTemplate + public void testAutoCreateTable(TestContainer container) + throws IOException, InterruptedException { + + TablePath tablePathSQL = TablePath.of("testauto", "dbo", "sqlserver_auto_create"); + TablePath tablePathSQL_Sql = TablePath.of("testauto", "dbo", "sqlserver_auto_create_sql"); + TablePath tablePathMySql = TablePath.of("auto", "sqlserver_auto_create_mysql"); + TablePath tablePathPG = TablePath.of("pg", "public", "sqlserver_auto_create_pg"); + TablePath tablePathOracle = TablePath.of("TESTUSER", "sqlserver_auto_create_oracle"); + + SqlServerCatalog sqlServerCatalog = + new SqlServerCatalog("sqlserver", "sa", password, sqlParse, "dbo"); + MySqlCatalog mySqlCatalog = new MySqlCatalog("mysql", "root", PASSWORD, MysqlUrlInfo); + PostgresCatalog postgresCatalog = + new PostgresCatalog("postgres", "testUser", PASSWORD, pg, "public"); + OracleCatalog oracleCatalog = + new OracleCatalog("oracle", "admin", "admin", oracle, "TESTUSER"); + mySqlCatalog.open(); + sqlServerCatalog.open(); + postgresCatalog.open(); + // oracleCatalog.open(); + + CatalogTable sqlServerCatalogTable = sqlServerCatalog.getTable(tablePathSQL); + + sqlServerCatalog.createTable(tablePathSQL_Sql, sqlServerCatalogTable, true); + postgresCatalog.createTable(tablePathPG, sqlServerCatalogTable, true); + // oracleCatalog.createTable(tablePathOracle, sqlServerCatalogTable, true); + mySqlCatalog.createTable(tablePathMySql, sqlServerCatalogTable, true); + + Assertions.assertTrue(checkMysql(mysqlCheck)); + // Assertions.assertTrue(checkOracle(oracleCheck)); + Assertions.assertTrue(checkSqlServer(sqlserverCheck)); + Assertions.assertTrue(checkPG(pgCheck)); + + // delete table + log.info("delete table"); + sqlServerCatalog.dropTable(tablePathSQL_Sql, true); + sqlServerCatalog.dropTable(tablePathSQL, true); + postgresCatalog.dropTable(tablePathPG, true); + // oracleCatalog.dropTable(tablePathOracle, true); + mySqlCatalog.dropTable(tablePathMySql, true); + + sqlServerCatalog.close(); + mySqlCatalog.close(); + postgresCatalog.close(); + } + + @Override + public void tearDown() throws Exception { + if (sqlserver_container != null) { + sqlserver_container.close(); + } + if (mysql_container != null) { + mysql_container.close(); + } + if (oracle_container != null) { + oracle_container.close(); + } + if (POSTGRESQL_CONTAINER != null) { + POSTGRESQL_CONTAINER.close(); + } + } + + private Connection getJdbcSqlServerConnection() throws SQLException { + return DriverManager.getConnection( + sqlserver_container.getJdbcUrl(), + sqlserver_container.getUsername(), + sqlserver_container.getPassword()); + } + + private Connection getJdbcMySqlConnection() throws SQLException { + return DriverManager.getConnection( + mysql_container.getJdbcUrl(), + mysql_container.getUsername(), + mysql_container.getPassword()); + } + + private Connection getJdbcPgConnection() throws SQLException { + return DriverManager.getConnection( + POSTGRESQL_CONTAINER.getJdbcUrl(), + POSTGRESQL_CONTAINER.getUsername(), + POSTGRESQL_CONTAINER.getPassword()); + } + + private Connection getJdbcOracleConnection() throws SQLException { + return DriverManager.getConnection( + oracle_container.getJdbcUrl(), + oracle_container.getUsername(), + oracle_container.getPassword()); + } + + private void initializeJdbcTable() { + try (Connection connection = getJdbcSqlServerConnection()) { + Statement statement = connection.createStatement(); + statement.execute(CREATE_DATABASE); + statement.execute(CREATE_TABLE_SQL); + statement.execute(getInsertSql); + // statement.executeBatch(); + } catch (SQLException e) { + throw new RuntimeException("Initializing PostgreSql table failed!", e); + } + } + + private boolean checkMysql(String sql) { + try (Connection connection = getJdbcMySqlConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + boolean tableExists = false; + if (resultSet.next()) { + tableExists = resultSet.getBoolean(1); + } + return tableExists; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private boolean checkPG(String sql) { + try (Connection connection = getJdbcPgConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + boolean tableExists = false; + if (resultSet.next()) { + tableExists = resultSet.getBoolean(1); + } + return tableExists; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private boolean checkSqlServer(String sql) { + try (Connection connection = getJdbcSqlServerConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + boolean tableExists = false; + if (resultSet.next()) { + tableExists = resultSet.getInt(1) == 1; + } + return tableExists; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private boolean checkOracle(String sql) { + try (Connection connection = getJdbcOracleConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + boolean tableExists = false; + if (resultSet.next()) { + tableExists = resultSet.getInt(1) == 1; + } + return tableExists; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/pom.xml index f803a4c61e8c..0b3e18bdbf25 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-jdbc-e2e/pom.xml @@ -31,6 +31,7 @@ connector-jdbc-e2e-part-1 connector-jdbc-e2e-part-2 connector-jdbc-e2e-part-3 + connector-jdbc-e2e-part-4 diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/CanalToKafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/CanalToKafkaIT.java index 0d8bb567ae4d..9afe0ce332f7 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/CanalToKafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/CanalToKafkaIT.java @@ -293,6 +293,17 @@ public void testCanalFormatKafkaCdcToPgsql(TestContainer container) Arrays.asList(107, "rocks", "box of assorted rocks", "7.88"), Arrays.asList(108, "jacket", "water resistent black wind breaker", "0.1")); Assertions.assertIterableEquals(expected, actual); + + try (Connection connection = + DriverManager.getConnection( + POSTGRESQL_CONTAINER.getJdbcUrl(), + POSTGRESQL_CONTAINER.getUsername(), + POSTGRESQL_CONTAINER.getPassword())) { + try (Statement statement = connection.createStatement()) { + statement.execute("truncate table sink"); + LOG.info("testCanalFormatKafkaCdcToPgsql truncate table sink"); + } + } } private void initKafkaConsumer() { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canalFormatIT/kafka_source_canal_cdc_to_pgsql.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canalFormatIT/kafka_source_canal_cdc_to_pgsql.conf index 9ce69a2344c3..2f7249dbdbdb 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canalFormatIT/kafka_source_canal_cdc_to_pgsql.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/canalFormatIT/kafka_source_canal_cdc_to_pgsql.conf @@ -47,13 +47,14 @@ source { sink { Jdbc { + driver = org.postgresql.Driver url = "jdbc:postgresql://postgresql:5432/test?loggerLevel=OFF" user = test password = test generate_sink_sql = true - database = public - table = sink + database = test + table = public.sink primary_keys = ["id"] } } \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-pulsar-e2e/src/test/java/org/apache/seatunnel/e2e/connector/pulsar/CanalToPulsarIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-pulsar-e2e/src/test/java/org/apache/seatunnel/e2e/connector/pulsar/CanalToPulsarIT.java index 716bd7dc90e6..ec8fd4813803 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-pulsar-e2e/src/test/java/org/apache/seatunnel/e2e/connector/pulsar/CanalToPulsarIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-pulsar-e2e/src/test/java/org/apache/seatunnel/e2e/connector/pulsar/CanalToPulsarIT.java @@ -337,5 +337,16 @@ void testCanalFormatMessages(TestContainer container) Arrays.asList(107, "rocks", "box of assorted rocks", "7.88"), Arrays.asList(108, "jacket", "water resistent black wind breaker", "0.1")); Assertions.assertIterableEquals(expected, actual); + + try (Connection connection = + DriverManager.getConnection( + POSTGRESQL_CONTAINER.getJdbcUrl(), + POSTGRESQL_CONTAINER.getUsername(), + POSTGRESQL_CONTAINER.getPassword())) { + try (Statement statement = connection.createStatement()) { + statement.execute("truncate table sink"); + LOG.info("testSinkCDCChangelog truncate table sink"); + } + } } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-pulsar-e2e/src/test/resources/cdc_canal_pulsar_to_pg.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-pulsar-e2e/src/test/resources/cdc_canal_pulsar_to_pg.conf index c287be676580..3ace667579e1 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-pulsar-e2e/src/test/resources/cdc_canal_pulsar_to_pg.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-pulsar-e2e/src/test/resources/cdc_canal_pulsar_to_pg.conf @@ -58,8 +58,8 @@ sink { user = test password = test generate_sink_sql = true - database = public - table = sink + database = test + table = public.sink primary_keys = ["id"] } } From a3c13e59eb30d603f326442bd784a0eb85f350a6 Mon Sep 17 00:00:00 2001 From: FlechazoW <35768015+FlechazoW@users.noreply.github.com> Date: Mon, 31 Jul 2023 17:33:36 +0800 Subject: [PATCH 34/43] [Feature][Connector V2][File] Add config of 'file_filter_pattern', which used for filtering files. (#5153) * [Feature][Connector V2][File] Add config of 'file_filter_pattern', which used for filtering files. --- docs/en/connector-v2/source/CosFile.md | 5 + docs/en/connector-v2/source/FtpFile.md | 1 + docs/en/connector-v2/source/HdfsFile.md | 5 + docs/en/connector-v2/source/LocalFile.md | 5 + docs/en/connector-v2/source/OssFile.md | 5 + docs/en/connector-v2/source/OssJindoFile.md | 5 + docs/en/connector-v2/source/S3File.md | 5 + docs/en/connector-v2/source/SftpFile.md | 5 + .../file/hdfs/source/BaseHdfsFileSource.java | 7 +- .../file/config/BaseSourceConfig.java | 7 + .../source/reader/AbstractReadStrategy.java | 30 +++- .../file/cos/source/CosFileSourceFactory.java | 1 + .../file/ftp/source/FtpFileSourceFactory.java | 1 + .../hdfs/source/HdfsFileSourceFactory.java | 1 + .../file/oss/source/OssFileSourceFactory.java | 1 + .../local/source/LocalFileSourceFactory.java | 1 + .../file/oss/source/OssFileSourceFactory.java | 1 + .../file/s3/source/S3FileSourceFactory.java | 1 + .../sftp/source/SftpFileSourceFactory.java | 1 + .../e2e/connector/file/ftp/FtpFileIT.java | 86 +++++------ .../excel/ftp_filter_excel_to_assert.conf | 141 ++++++++++++++++++ .../e2e/connector/file/local/LocalFileIT.java | 119 ++++++--------- .../excel/local_filter_excel_to_assert.conf | 131 ++++++++++++++++ .../e2e/connector/file/fstp/SftpFileIT.java | 78 +++++----- .../excel/sftp_filter_excel_to_assert.conf | 132 ++++++++++++++++ .../e2e/common/container/TestHelper.java | 40 +++++ .../e2e/common/util/ContainerUtil.java | 6 + 27 files changed, 649 insertions(+), 172 deletions(-) create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/excel/ftp_filter_excel_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/excel/local_filter_excel_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-sftp-e2e/src/test/resources/excel/sftp_filter_excel_to_assert.conf create mode 100644 seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestHelper.java diff --git a/docs/en/connector-v2/source/CosFile.md b/docs/en/connector-v2/source/CosFile.md index 18fc0299c9e2..dd1e77ebcfd0 100644 --- a/docs/en/connector-v2/source/CosFile.md +++ b/docs/en/connector-v2/source/CosFile.md @@ -55,6 +55,7 @@ Read all the data in a split in a pollNext call. What splits are read will be sa | schema | config | no | - | | common-options | | no | - | | sheet_name | string | no | - | +| file_filter_pattern | string | no | - | ### path [string] @@ -247,6 +248,10 @@ Source plugin common parameters, please refer to [Source Common Options](common- Reader the sheet of the workbook,Only used when file_format is excel. +### file_filter_pattern [string] + +Filter pattern, which used for filtering files. + ## Example ```hocon diff --git a/docs/en/connector-v2/source/FtpFile.md b/docs/en/connector-v2/source/FtpFile.md index b550bde8baac..6737511e63db 100644 --- a/docs/en/connector-v2/source/FtpFile.md +++ b/docs/en/connector-v2/source/FtpFile.md @@ -48,6 +48,7 @@ If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you | schema | config | no | - | | common-options | | no | - | | sheet_name | string | no | - | +| file_filter_pattern | string | no | - | ### host [string] diff --git a/docs/en/connector-v2/source/HdfsFile.md b/docs/en/connector-v2/source/HdfsFile.md index d255f4fd3a7c..1d285c539a3f 100644 --- a/docs/en/connector-v2/source/HdfsFile.md +++ b/docs/en/connector-v2/source/HdfsFile.md @@ -53,6 +53,7 @@ Read all the data in a split in a pollNext call. What splits are read will be sa | schema | config | no | - | | common-options | | no | - | | sheet_name | string | no | - | +| file_filter_pattern | string | no | - | ### path [string] @@ -245,6 +246,10 @@ Source plugin common parameters, please refer to [Source Common Options](common- Reader the sheet of the workbook,Only used when file_format is excel. +### file_filter_pattern [string] + +Filter pattern, which used for filtering files. + ## Example ```hocon diff --git a/docs/en/connector-v2/source/LocalFile.md b/docs/en/connector-v2/source/LocalFile.md index d33288b7a57a..7d80a4d2b7a3 100644 --- a/docs/en/connector-v2/source/LocalFile.md +++ b/docs/en/connector-v2/source/LocalFile.md @@ -49,6 +49,7 @@ Read all the data in a split in a pollNext call. What splits are read will be sa | schema | config | no | - | | common-options | | no | - | | sheet_name | string | no | - | +| file_filter_pattern | string | no | - | ### path [string] @@ -225,6 +226,10 @@ Source plugin common parameters, please refer to [Source Common Options](common- Reader the sheet of the workbook,Only used when file_format is excel. +### file_filter_pattern [string] + +Filter pattern, which used for filtering files. + ## Example ```hocon diff --git a/docs/en/connector-v2/source/OssFile.md b/docs/en/connector-v2/source/OssFile.md index 532b4d03aa79..12f2141cd6ed 100644 --- a/docs/en/connector-v2/source/OssFile.md +++ b/docs/en/connector-v2/source/OssFile.md @@ -56,6 +56,7 @@ Read all the data in a split in a pollNext call. What splits are read will be sa | schema | config | no | - | | common-options | | no | - | | sheet_name | string | no | - | +| file_filter_pattern | string | no | - | ### path [string] @@ -282,6 +283,10 @@ Reader the sheet of the workbook,Only used when file_format is excel. ``` +### file_filter_pattern [string] + +Filter pattern, which used for filtering files. + ## Changelog ### 2.2.0-beta 2022-09-26 diff --git a/docs/en/connector-v2/source/OssJindoFile.md b/docs/en/connector-v2/source/OssJindoFile.md index 3e3649e19b92..913d277683e4 100644 --- a/docs/en/connector-v2/source/OssJindoFile.md +++ b/docs/en/connector-v2/source/OssJindoFile.md @@ -56,6 +56,7 @@ Read all the data in a split in a pollNext call. What splits are read will be sa | schema | config | no | - | | common-options | | no | - | | sheet_name | string | no | - | +| file_filter_pattern | string | no | - | ### path [string] @@ -248,6 +249,10 @@ Source plugin common parameters, please refer to [Source Common Options](common- Reader the sheet of the workbook,Only used when file_format is excel. +### file_filter_pattern [string] + +Filter pattern, which used for filtering files. + ## Example ```hocon diff --git a/docs/en/connector-v2/source/S3File.md b/docs/en/connector-v2/source/S3File.md index f58a1a6bc36e..79a89be1c27f 100644 --- a/docs/en/connector-v2/source/S3File.md +++ b/docs/en/connector-v2/source/S3File.md @@ -57,6 +57,7 @@ Read all the data in a split in a pollNext call. What splits are read will be sa | schema | config | no | - | | common-options | | no | - | | sheet_name | string | no | - | +| file_filter_pattern | string | no | - | ### path [string] @@ -299,6 +300,10 @@ Reader the sheet of the workbook,Only used when file_format is excel. ``` +### file_filter_pattern [string] + +Filter pattern, which used for filtering files. + ## Changelog ### 2.3.0-beta 2022-10-20 diff --git a/docs/en/connector-v2/source/SftpFile.md b/docs/en/connector-v2/source/SftpFile.md index 500ec2af5b57..22047d481ed6 100644 --- a/docs/en/connector-v2/source/SftpFile.md +++ b/docs/en/connector-v2/source/SftpFile.md @@ -47,6 +47,7 @@ If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you | schema | config | no | - | | common-options | | no | - | | sheet_name | string | no | - | +| file_filter_pattern | string | no | - | ### host [string] @@ -226,6 +227,10 @@ Source plugin common parameters, please refer to [Source Common Options](common- Reader the sheet of the workbook,Only used when file_format is excel. +### file_filter_pattern [string] + +Filter pattern, which used for filtering files. + ## Example ```hocon diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/BaseHdfsFileSource.java b/seatunnel-connectors-v2/connector-file/connector-file-base-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/BaseHdfsFileSource.java index 9864fc3750b6..57d2ceca6eb4 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/BaseHdfsFileSource.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/BaseHdfsFileSource.java @@ -79,12 +79,7 @@ public void prepare(Config pluginConfig) throws PrepareFailException { throw new FileConnectorException( FileConnectorErrorCode.FILE_LIST_GET_FAILED, errorMsg, e); } - if (filePaths.isEmpty()) { - throw new FileConnectorException( - FileConnectorErrorCode.FILE_LIST_EMPTY, - "The target file list is empty," - + "SeaTunnel will not be able to sync empty table"); - } + // support user-defined schema FileFormat fileFormat = FileFormat.valueOf( diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseSourceConfig.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseSourceConfig.java index fa65628bd561..7b1e32d1dad7 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseSourceConfig.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/config/BaseSourceConfig.java @@ -112,4 +112,11 @@ public class BaseSourceConfig { .stringType() .noDefaultValue() .withDescription("To be read sheet name,only valid for excel files"); + + public static final Option FILE_FILTER_PATTERN = + Options.key("file_filter_pattern") + .stringType() + .noDefaultValue() + .withDescription( + "File pattern. The connector will filter some files base on the pattern."); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java index ea6c902c05a9..e4e1694f30dc 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-base/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/source/reader/AbstractReadStrategy.java @@ -24,6 +24,8 @@ import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.seatunnel.file.config.BaseSourceConfig; import org.apache.seatunnel.connectors.seatunnel.file.config.HadoopConf; +import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorErrorCode; +import org.apache.seatunnel.connectors.seatunnel.file.exception.FileConnectorException; import org.apache.seatunnel.connectors.seatunnel.file.sink.util.FileSystemUtils; import org.apache.hadoop.conf.Configuration; @@ -43,6 +45,9 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import static org.apache.parquet.avro.AvroReadSupport.READ_INT96_AS_FIXED; import static org.apache.parquet.avro.AvroSchemaConverter.ADD_LIST_ELEMENT_RECORDS; @@ -74,6 +79,8 @@ public abstract class AbstractReadStrategy implements ReadStrategy { protected long skipHeaderNumber = BaseSourceConfig.SKIP_HEADER_ROW_NUMBER.defaultValue(); protected transient boolean isKerberosAuthorization = false; + protected Pattern pattern; + @Override public void init(HadoopConf conf) { this.hadoopConf = conf; @@ -126,7 +133,7 @@ public List getFileNamesByPath(HadoopConf hadoopConf, String path) throw fileNames.addAll(getFileNamesByPath(hadoopConf, fileStatus.getPath().toString())); continue; } - if (fileStatus.isFile()) { + if (fileStatus.isFile() && filterFileByPattern(fileStatus)) { // filter '_SUCCESS' file if (!fileStatus.getPath().getName().equals("_SUCCESS") && !fileStatus.getPath().getName().startsWith(".")) { @@ -146,6 +153,15 @@ public List getFileNamesByPath(HadoopConf hadoopConf, String path) throw } } } + + if (fileNames.isEmpty()) { + throw new FileConnectorException( + FileConnectorErrorCode.FILE_LIST_EMPTY, + "The target file list is empty," + + "SeaTunnel will not be able to sync empty table, " + + "please check the configuration parameters such as: [file_filter_pattern]"); + } + return fileNames; } @@ -166,6 +182,11 @@ public void setPluginConfig(Config pluginConfig) { if (pluginConfig.hasPath(BaseSourceConfig.READ_COLUMNS.key())) { readColumns.addAll(pluginConfig.getStringList(BaseSourceConfig.READ_COLUMNS.key())); } + if (pluginConfig.hasPath(BaseSourceConfig.FILE_FILTER_PATTERN.key())) { + String filterPattern = + pluginConfig.getString(BaseSourceConfig.FILE_FILTER_PATTERN.key()); + this.pattern = Pattern.compile(Matcher.quoteReplacement(filterPattern)); + } } @Override @@ -214,4 +235,11 @@ protected SeaTunnelRowType mergePartitionTypes(String path, SeaTunnelRowType sea // return merge row type return new SeaTunnelRowType(newFieldNames, newFieldTypes); } + + protected boolean filterFileByPattern(FileStatus fileStatus) { + if (Objects.nonNull(pattern)) { + return pattern.matcher(fileStatus.getPath().getName()).matches(); + } + return true; + } } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java index d0b781f1a144..496e9277f4e3 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-cos/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/cos/source/CosFileSourceFactory.java @@ -60,6 +60,7 @@ public OptionRule optionRule() { .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) .optional(BaseSourceConfig.TIME_FORMAT) + .optional(BaseSourceConfig.FILE_FILTER_PATTERN) .build(); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSourceFactory.java index d2d11da5b467..4ab637c43484 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-ftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/ftp/source/FtpFileSourceFactory.java @@ -60,6 +60,7 @@ public OptionRule optionRule() { .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) .optional(BaseSourceConfig.TIME_FORMAT) + .optional(BaseSourceConfig.FILE_FILTER_PATTERN) .build(); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSourceFactory.java index d4c17384904e..c3d406d62c79 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-hadoop/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/hdfs/source/HdfsFileSourceFactory.java @@ -57,6 +57,7 @@ public OptionRule optionRule() { .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) .optional(BaseSourceConfig.TIME_FORMAT) + .optional(BaseSourceConfig.FILE_FILTER_PATTERN) .build(); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java index 502567676254..eaea7bccb61e 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-jindo-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java @@ -60,6 +60,7 @@ public OptionRule optionRule() { .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) .optional(BaseSourceConfig.TIME_FORMAT) + .optional(BaseSourceConfig.FILE_FILTER_PATTERN) .build(); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSourceFactory.java index 4ae2ae3a9b61..03ec8660ce2d 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-local/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/local/source/LocalFileSourceFactory.java @@ -56,6 +56,7 @@ public OptionRule optionRule() { .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) .optional(BaseSourceConfig.TIME_FORMAT) + .optional(BaseSourceConfig.FILE_FILTER_PATTERN) .build(); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java index c6a2d7040923..e7d862bd44ad 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-oss/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/oss/source/OssFileSourceFactory.java @@ -60,6 +60,7 @@ public OptionRule optionRule() { .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) .optional(BaseSourceConfig.TIME_FORMAT) + .optional(BaseSourceConfig.FILE_FILTER_PATTERN) .build(); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java index 71156a21b66c..a3b48088650c 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-s3/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/s3/source/S3FileSourceFactory.java @@ -65,6 +65,7 @@ public OptionRule optionRule() { .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) .optional(BaseSourceConfig.TIME_FORMAT) + .optional(BaseSourceConfig.FILE_FILTER_PATTERN) .build(); } diff --git a/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSourceFactory.java b/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSourceFactory.java index 18cda2fbe5ec..e9efe1cdf9b3 100644 --- a/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSourceFactory.java +++ b/seatunnel-connectors-v2/connector-file/connector-file-sftp/src/main/java/org/apache/seatunnel/connectors/seatunnel/file/sftp/source/SftpFileSourceFactory.java @@ -60,6 +60,7 @@ public OptionRule optionRule() { .optional(BaseSourceConfig.DATE_FORMAT) .optional(BaseSourceConfig.DATETIME_FORMAT) .optional(BaseSourceConfig.TIME_FORMAT) + .optional(BaseSourceConfig.FILE_FILTER_PATTERN) .build(); } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/ftp/FtpFileIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/ftp/FtpFileIT.java index 5fc0e486091c..15a58ebf082f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/ftp/FtpFileIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/ftp/FtpFileIT.java @@ -21,23 +21,20 @@ import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.EngineType; import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.container.TestHelper; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.e2e.common.util.ContainerUtil; import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; -import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.MountableFile; import lombok.extern.slf4j.Slf4j; import java.io.IOException; -import java.nio.file.Path; import java.util.Collections; import java.util.stream.Stream; @@ -87,19 +84,26 @@ public void startUp() throws Exception { Startables.deepStart(Stream.of(ftpContainer)).join(); log.info("ftp container started"); - Path jsonPath = ContainerUtil.getResourcesFile("/json/e2e.json").toPath(); - Path textPath = ContainerUtil.getResourcesFile("/text/e2e.txt").toPath(); - Path excelPath = ContainerUtil.getResourcesFile("/excel/e2e.xlsx").toPath(); - - ftpContainer.copyFileToContainer( - MountableFile.forHostPath(jsonPath), - "/home/vsftpd/seatunnel/tmp/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json"); - ftpContainer.copyFileToContainer( - MountableFile.forHostPath(textPath), - "/home/vsftpd/seatunnel/tmp/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt"); - ftpContainer.copyFileToContainer( - MountableFile.forHostPath(excelPath), - "/home/vsftpd/seatunnel/tmp/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx"); + ContainerUtil.copyFileIntoContainers( + "/json/e2e.json", + "/home/vsftpd/seatunnel/tmp/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json", + ftpContainer); + + ContainerUtil.copyFileIntoContainers( + "/text/e2e.txt", + "/home/vsftpd/seatunnel/tmp/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt", + ftpContainer); + + ContainerUtil.copyFileIntoContainers( + "/excel/e2e.xlsx", + "/home/vsftpd/seatunnel/tmp/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx", + ftpContainer); + + ContainerUtil.copyFileIntoContainers( + "/excel/e2e.xlsx", + "/home/vsftpd/seatunnel/tmp/seatunnel/read/excel_filter/name=tyrantlucifer/hobby=coding/e2e_filter.xlsx", + ftpContainer); + ftpContainer.execInContainer("sh", "-c", "chmod -R 777 /home/vsftpd/seatunnel/"); ftpContainer.execInContainer("sh", "-c", "chown -R ftp:ftp /home/vsftpd/seatunnel/"); } @@ -107,51 +111,31 @@ public void startUp() throws Exception { @TestTemplate public void testFtpFileReadAndWrite(TestContainer container) throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); // test write ftp excel file - Container.ExecResult excelWriteResult = - container.executeJob("/excel/fake_source_to_ftp_excel.conf"); - Assertions.assertEquals(0, excelWriteResult.getExitCode(), excelWriteResult.getStderr()); + helper.execute("/excel/fake_source_to_ftp_excel.conf"); // test read ftp excel file - Container.ExecResult excelReadResult = - container.executeJob("/excel/ftp_excel_to_assert.conf"); - Assertions.assertEquals(0, excelReadResult.getExitCode(), excelReadResult.getStderr()); + helper.execute("/excel/ftp_excel_to_assert.conf"); // test read ftp excel file with projection - Container.ExecResult excelProjectionReadResult = - container.executeJob("/excel/ftp_excel_projection_to_assert.conf"); - Assertions.assertEquals( - 0, excelProjectionReadResult.getExitCode(), excelProjectionReadResult.getStderr()); + helper.execute("/excel/ftp_excel_projection_to_assert.conf"); + // test read ftp excel file with filter + helper.execute("/excel/ftp_filter_excel_to_assert.conf"); // test write ftp text file - Container.ExecResult textWriteResult = - container.executeJob("/text/fake_to_ftp_file_text.conf"); - Assertions.assertEquals(0, textWriteResult.getExitCode()); + helper.execute("/text/fake_to_ftp_file_text.conf"); // test read skip header - Container.ExecResult textWriteAndSkipResult = - container.executeJob("/text/ftp_file_text_skip_headers.conf"); - Assertions.assertEquals(0, textWriteAndSkipResult.getExitCode()); + helper.execute("/text/ftp_file_text_skip_headers.conf"); // test read ftp text file - Container.ExecResult textReadResult = - container.executeJob("/text/ftp_file_text_to_assert.conf"); - Assertions.assertEquals(0, textReadResult.getExitCode()); + helper.execute("/text/ftp_file_text_to_assert.conf"); // test read ftp text file with projection - Container.ExecResult textProjectionResult = - container.executeJob("/text/ftp_file_text_projection_to_assert.conf"); - Assertions.assertEquals(0, textProjectionResult.getExitCode()); + helper.execute("/text/ftp_file_text_projection_to_assert.conf"); // test write ftp json file - Container.ExecResult jsonWriteResult = - container.executeJob("/json/fake_to_ftp_file_json.conf"); - Assertions.assertEquals(0, jsonWriteResult.getExitCode()); + helper.execute("/json/fake_to_ftp_file_json.conf"); // test read ftp json file - Container.ExecResult jsonReadResult = - container.executeJob("/json/ftp_file_json_to_assert.conf"); - Assertions.assertEquals(0, jsonReadResult.getExitCode()); + helper.execute("/json/ftp_file_json_to_assert.conf"); // test write ftp parquet file - Container.ExecResult parquetWriteResult = - container.executeJob("/parquet/fake_to_ftp_file_parquet.conf"); - Assertions.assertEquals(0, parquetWriteResult.getExitCode()); + helper.execute("/parquet/fake_to_ftp_file_parquet.conf"); // test write ftp orc file - Container.ExecResult orcWriteResult = - container.executeJob("/orc/fake_to_ftp_file_orc.conf"); - Assertions.assertEquals(0, orcWriteResult.getExitCode()); + helper.execute("/orc/fake_to_ftp_file_orc.conf"); } @AfterAll diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/excel/ftp_filter_excel_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/excel/ftp_filter_excel_to_assert.conf new file mode 100644 index 000000000000..6af42f6f3d62 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-ftp-e2e/src/test/resources/excel/ftp_filter_excel_to_assert.conf @@ -0,0 +1,141 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + # You can set flink configuration here + execution.parallelism = 1 + job.mode = "BATCH" + + # You can set spark configuration here + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FtpFile { + host = "ftp" + port = 21 + user = seatunnel + password = pass + path = "/tmp/seatunnel/read/excel_filter" + result_table_name = "ftp" + file_format_type = excel + delimiter = ; + skip_header_row_number = 1 + file_filter_pattern = "e2e_filter.*" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + + +sink { + Assert { + source_table_name = "ftp" + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/local/LocalFileIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/local/LocalFileIT.java index f5c220deabd2..aed357672638 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/local/LocalFileIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/local/LocalFileIT.java @@ -21,17 +21,14 @@ import org.apache.seatunnel.e2e.common.container.ContainerExtendedFactory; import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.seatunnel.e2e.common.container.TestContainerId; +import org.apache.seatunnel.e2e.common.container.TestHelper; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.e2e.common.junit.TestContainerExtension; import org.apache.seatunnel.e2e.common.util.ContainerUtil; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.TestTemplate; -import org.testcontainers.containers.Container; -import org.testcontainers.utility.MountableFile; import java.io.IOException; -import java.nio.file.Path; @DisabledOnContainer( value = {TestContainerId.SPARK_2_4}, @@ -43,88 +40,70 @@ public class LocalFileIT extends TestSuiteBase { @TestContainerExtension private final ContainerExtendedFactory extendedFactory = container -> { - Path jsonPath = ContainerUtil.getResourcesFile("/json/e2e.json").toPath(); - Path orcPath = ContainerUtil.getResourcesFile("/orc/e2e.orc").toPath(); - Path parquetPath = ContainerUtil.getResourcesFile("/parquet/e2e.parquet").toPath(); - Path textPath = ContainerUtil.getResourcesFile("/text/e2e.txt").toPath(); - Path excelPath = ContainerUtil.getResourcesFile("/excel/e2e.xlsx").toPath(); - container.copyFileToContainer( - MountableFile.forHostPath(jsonPath), - "/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json"); - container.copyFileToContainer( - MountableFile.forHostPath(orcPath), - "/seatunnel/read/orc/name=tyrantlucifer/hobby=coding/e2e.orc"); - container.copyFileToContainer( - MountableFile.forHostPath(parquetPath), - "/seatunnel/read/parquet/name=tyrantlucifer/hobby=coding/e2e.parquet"); - container.copyFileToContainer( - MountableFile.forHostPath(textPath), - "/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt"); - container.copyFileToContainer( - MountableFile.forHostPath(excelPath), - "/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx"); + ContainerUtil.copyFileIntoContainers( + "/json/e2e.json", + "/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json", + container); + + ContainerUtil.copyFileIntoContainers( + "/text/e2e.txt", + "/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt", + container); + + ContainerUtil.copyFileIntoContainers( + "/excel/e2e.xlsx", + "/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx", + container); + + ContainerUtil.copyFileIntoContainers( + "/orc/e2e.orc", + "/seatunnel/read/orc/name=tyrantlucifer/hobby=coding/e2e.orc", + container); + + ContainerUtil.copyFileIntoContainers( + "/parquet/e2e.parquet", + "/seatunnel/read/parquet/name=tyrantlucifer/hobby=coding/e2e.parquet", + container); + + ContainerUtil.copyFileIntoContainers( + "/excel/e2e.xlsx", + "/seatunnel/read/excel_filter/name=tyrantlucifer/hobby=coding/e2e_filter.xlsx", + container); }; @TestTemplate public void testLocalFileReadAndWrite(TestContainer container) throws IOException, InterruptedException { - Container.ExecResult excelWriteResult = - container.executeJob("/excel/fake_to_local_excel.conf"); - Assertions.assertEquals(0, excelWriteResult.getExitCode(), excelWriteResult.getStderr()); - Container.ExecResult excelReadResult = - container.executeJob("/excel/local_excel_to_assert.conf"); - Assertions.assertEquals(0, excelReadResult.getExitCode(), excelReadResult.getStderr()); - Container.ExecResult excelProjectionReadResult = - container.executeJob("/excel/local_excel_projection_to_assert.conf"); - Assertions.assertEquals( - 0, excelProjectionReadResult.getExitCode(), excelProjectionReadResult.getStderr()); + TestHelper helper = new TestHelper(container); + + helper.execute("/excel/fake_to_local_excel.conf"); + helper.execute("/excel/local_excel_to_assert.conf"); + helper.execute("/excel/local_excel_projection_to_assert.conf"); // test write local text file - Container.ExecResult textWriteResult = - container.executeJob("/text/fake_to_local_file_text.conf"); - Assertions.assertEquals(0, textWriteResult.getExitCode()); + helper.execute("/text/fake_to_local_file_text.conf"); // test read skip header - Container.ExecResult textWriteAndSkipResult = - container.executeJob("/text/local_file_text_skip_headers.conf"); - Assertions.assertEquals(0, textWriteAndSkipResult.getExitCode()); + helper.execute("/text/local_file_text_skip_headers.conf"); // test read local text file - Container.ExecResult textReadResult = - container.executeJob("/text/local_file_text_to_assert.conf"); - Assertions.assertEquals(0, textReadResult.getExitCode()); + helper.execute("/text/local_file_text_to_assert.conf"); // test read local text file with projection - Container.ExecResult textProjectionResult = - container.executeJob("/text/local_file_text_projection_to_assert.conf"); - Assertions.assertEquals(0, textProjectionResult.getExitCode()); + helper.execute("/text/local_file_text_projection_to_assert.conf"); // test write local json file - Container.ExecResult jsonWriteResult = - container.executeJob("/json/fake_to_local_file_json.conf"); - Assertions.assertEquals(0, jsonWriteResult.getExitCode()); + helper.execute("/json/fake_to_local_file_json.conf"); // test read local json file - Container.ExecResult jsonReadResult = - container.executeJob("/json/local_file_json_to_assert.conf"); - Assertions.assertEquals(0, jsonReadResult.getExitCode()); + helper.execute("/json/local_file_json_to_assert.conf"); // test write local orc file - Container.ExecResult orcWriteResult = - container.executeJob("/orc/fake_to_local_file_orc.conf"); - Assertions.assertEquals(0, orcWriteResult.getExitCode()); + helper.execute("/orc/fake_to_local_file_orc.conf"); // test read local orc file - Container.ExecResult orcReadResult = - container.executeJob("/orc/local_file_orc_to_assert.conf"); - Assertions.assertEquals(0, orcReadResult.getExitCode()); + helper.execute("/orc/local_file_orc_to_assert.conf"); // test read local orc file with projection - Container.ExecResult orcProjectionResult = - container.executeJob("/orc/local_file_orc_projection_to_assert.conf"); - Assertions.assertEquals(0, orcProjectionResult.getExitCode()); + helper.execute("/orc/local_file_orc_projection_to_assert.conf"); // test write local parquet file - Container.ExecResult parquetWriteResult = - container.executeJob("/parquet/fake_to_local_file_parquet.conf"); - Assertions.assertEquals(0, parquetWriteResult.getExitCode()); + helper.execute("/parquet/fake_to_local_file_parquet.conf"); // test read local parquet file - Container.ExecResult parquetReadResult = - container.executeJob("/parquet/local_file_parquet_to_assert.conf"); - Assertions.assertEquals(0, parquetReadResult.getExitCode()); + helper.execute("/parquet/local_file_parquet_to_assert.conf"); // test read local parquet file with projection - Container.ExecResult parquetProjectionResult = - container.executeJob("/parquet/local_file_parquet_projection_to_assert.conf"); - Assertions.assertEquals(0, parquetProjectionResult.getExitCode()); + helper.execute("/parquet/local_file_parquet_projection_to_assert.conf"); + // test read filtered local file + helper.execute("/excel/local_filter_excel_to_assert.conf"); } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/excel/local_filter_excel_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/excel/local_filter_excel_to_assert.conf new file mode 100644 index 000000000000..86039b44dbfe --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-local-e2e/src/test/resources/excel/local_filter_excel_to_assert.conf @@ -0,0 +1,131 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + execution.parallelism = 1 + spark.app.name = "SeaTunnel" + spark.executor.instances = 2 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local + job.mode = "BATCH" +} + +source { + LocalFile { + path = "/seatunnel/read/excel_filter" + result_table_name = "fake" + file_format_type = excel + delimiter = ; + skip_header_row_number = 1 + file_filter_pattern = "e2e_filter.*" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Assert { + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = name + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = hobby + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-sftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/fstp/SftpFileIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-sftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/fstp/SftpFileIT.java index 82d1be73db16..e5fbcb5f5ef1 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-sftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/fstp/SftpFileIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-sftp-e2e/src/test/java/org/apache/seatunnel/e2e/connector/file/fstp/SftpFileIT.java @@ -21,22 +21,19 @@ import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.TestContainer; import org.apache.seatunnel.e2e.common.container.TestContainerId; +import org.apache.seatunnel.e2e.common.container.TestHelper; import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; import org.apache.seatunnel.e2e.common.util.ContainerUtil; import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; -import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; import org.testcontainers.lifecycle.Startables; -import org.testcontainers.utility.MountableFile; import lombok.extern.slf4j.Slf4j; import java.io.IOException; -import java.nio.file.Path; import java.util.Collections; import java.util.stream.Stream; @@ -75,61 +72,54 @@ public void startUp() throws Exception { sftpContainer.start(); Startables.deepStart(Stream.of(sftpContainer)).join(); log.info("Sftp container started"); - Path jsonPath = ContainerUtil.getResourcesFile("/json/e2e.json").toPath(); - Path textPath = ContainerUtil.getResourcesFile("/text/e2e.txt").toPath(); - Path excelPath = ContainerUtil.getResourcesFile("/excel/e2e.xlsx").toPath(); - sftpContainer.copyFileToContainer( - MountableFile.forHostPath(jsonPath), - "/home/seatunnel/tmp/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json"); - sftpContainer.copyFileToContainer( - MountableFile.forHostPath(textPath), - "/home/seatunnel/tmp/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt"); - sftpContainer.copyFileToContainer( - MountableFile.forHostPath(excelPath), - "/home/seatunnel/tmp/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx"); + + ContainerUtil.copyFileIntoContainers( + "/json/e2e.json", + "/home/seatunnel/tmp/seatunnel/read/json/name=tyrantlucifer/hobby=coding/e2e.json", + sftpContainer); + + ContainerUtil.copyFileIntoContainers( + "/text/e2e.txt", + "/home/seatunnel/tmp/seatunnel/read/text/name=tyrantlucifer/hobby=coding/e2e.txt", + sftpContainer); + + ContainerUtil.copyFileIntoContainers( + "/excel/e2e.xlsx", + "/home/seatunnel/tmp/seatunnel/read/excel/name=tyrantlucifer/hobby=coding/e2e.xlsx", + sftpContainer); + + ContainerUtil.copyFileIntoContainers( + "/excel/e2e.xlsx", + "/home/seatunnel/tmp/seatunnel/read/excel_filter/name=tyrantlucifer/hobby=coding/e2e_filter.xlsx", + sftpContainer); + sftpContainer.execInContainer("sh", "-c", "chown -R seatunnel /home/seatunnel/tmp/"); } @TestTemplate public void testSftpFileReadAndWrite(TestContainer container) throws IOException, InterruptedException { + TestHelper helper = new TestHelper(container); // test write sftp excel file - Container.ExecResult excelWriteResult = - container.executeJob("/excel/fakesource_to_sftp_excel.conf"); - Assertions.assertEquals(0, excelWriteResult.getExitCode(), excelWriteResult.getStderr()); + helper.execute("/excel/fakesource_to_sftp_excel.conf"); // test read sftp excel file - Container.ExecResult excelReadResult = - container.executeJob("/excel/sftp_excel_to_assert.conf"); - Assertions.assertEquals(0, excelReadResult.getExitCode(), excelReadResult.getStderr()); + helper.execute("/excel/sftp_excel_to_assert.conf"); // test read sftp excel file with projection - Container.ExecResult excelProjectionReadResult = - container.executeJob("/excel/sftp_excel_projection_to_assert.conf"); - Assertions.assertEquals( - 0, excelProjectionReadResult.getExitCode(), excelProjectionReadResult.getStderr()); + helper.execute("/excel/sftp_excel_projection_to_assert.conf"); + // test read sftp excel file with filter pattern + helper.execute("/excel/sftp_filter_excel_to_assert.conf"); // test write sftp text file - Container.ExecResult textWriteResult = - container.executeJob("/text/fake_to_sftp_file_text.conf"); - Assertions.assertEquals(0, textWriteResult.getExitCode()); + helper.execute("/text/fake_to_sftp_file_text.conf"); // test read skip header - Container.ExecResult textWriteAndSkipResult = - container.executeJob("/text/sftp_file_text_skip_headers.conf"); - Assertions.assertEquals(0, textWriteAndSkipResult.getExitCode()); + helper.execute("/text/sftp_file_text_skip_headers.conf"); // test read sftp text file - Container.ExecResult textReadResult = - container.executeJob("/text/sftp_file_text_to_assert.conf"); - Assertions.assertEquals(0, textReadResult.getExitCode()); + helper.execute("/text/sftp_file_text_to_assert.conf"); // test read sftp text file with projection - Container.ExecResult textProjectionResult = - container.executeJob("/text/sftp_file_text_projection_to_assert.conf"); - Assertions.assertEquals(0, textProjectionResult.getExitCode()); + helper.execute("/text/sftp_file_text_projection_to_assert.conf"); // test write sftp json file - Container.ExecResult jsonWriteResult = - container.executeJob("/json/fake_to_sftp_file_json.conf"); - Assertions.assertEquals(0, jsonWriteResult.getExitCode()); + helper.execute("/json/fake_to_sftp_file_json.conf"); // test read sftp json file - Container.ExecResult jsonReadResult = - container.executeJob("/json/sftp_file_json_to_assert.conf"); - Assertions.assertEquals(0, jsonReadResult.getExitCode()); + helper.execute("/json/sftp_file_json_to_assert.conf"); } @AfterAll diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-sftp-e2e/src/test/resources/excel/sftp_filter_excel_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-sftp-e2e/src/test/resources/excel/sftp_filter_excel_to_assert.conf new file mode 100644 index 000000000000..b6cd92f712a5 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-file-sftp-e2e/src/test/resources/excel/sftp_filter_excel_to_assert.conf @@ -0,0 +1,132 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +env { + # You can set flink configuration here + execution.parallelism = 1 + job.mode = "BATCH" + + # You can set spark configuration here + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + SftpFile { + path = "tmp/seatunnel/read/excel_filter" + result_table_name = "sftp" + file_format_type = excel + host = "sftp" + port = 22 + user = seatunnel + password = pass + delimiter = ";" + file_filter_pattern = "e2e_filter.*" + skip_header_row_number = 1 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_tinyint = tinyint + c_smallint = smallint + c_int = int + c_bigint = bigint + c_float = float + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Assert { + source_table_name = "sftp" + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + } + ], + field_rules = [ + { + field_name = c_string + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_boolean + field_type = boolean + field_value = [ + { + rule_type = NOT_NULL + } + ] + }, + { + field_name = c_double + field_type = double + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } + } +} + + + + + + + + + + + diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestHelper.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestHelper.java new file mode 100644 index 000000000000..a88723f82012 --- /dev/null +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/container/TestHelper.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.common.container; + +import org.junit.jupiter.api.Assertions; +import org.testcontainers.containers.Container; + +import java.io.IOException; + +public class TestHelper { + private final TestContainer container; + + public TestHelper(TestContainer container) { + this.container = container; + } + + public void execute(String file) throws IOException, InterruptedException { + execute(0, file); + } + + public void execute(int exceptResult, String file) throws IOException, InterruptedException { + Container.ExecResult result = container.executeJob(file); + Assertions.assertEquals(exceptResult, result.getExitCode(), result.getStderr()); + } +} diff --git a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/util/ContainerUtil.java b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/util/ContainerUtil.java index 92d6100a7cea..fa5660a17002 100644 --- a/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/util/ContainerUtil.java +++ b/seatunnel-e2e/seatunnel-e2e-common/src/test/java/org/apache/seatunnel/e2e/common/util/ContainerUtil.java @@ -246,4 +246,10 @@ public static List discoverTestContainers() { throw new FactoryException("Could not load service provider for containers.", e); } } + + public static void copyFileIntoContainers( + String fileName, String targetPath, GenericContainer container) { + Path path = getResourcesFile(fileName).toPath(); + container.copyFileToContainer(MountableFile.forHostPath(path), targetPath); + } } From b0815f2a95e01acedb1fde7d56a81b32a5d3b494 Mon Sep 17 00:00:00 2001 From: kk <127465317+jackyyyyyssss@users.noreply.github.com> Date: Mon, 31 Jul 2023 19:30:13 +0800 Subject: [PATCH 35/43] [Improve][Connector-v2][Jdbc] check url not null throw friendly message (#5097) * check url not null throw friendly message * check jdbc source config * modify jdbc validate method --------- Co-authored-by: 80597928 Co-authored-by: 80597928 <673421862@qq.com> --- .../seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java index aa001f78e2a6..1bf1b332fa98 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/source/JdbcSource.java @@ -21,6 +21,7 @@ import org.apache.seatunnel.api.common.PrepareFailException; import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.configuration.util.ConfigValidator; import org.apache.seatunnel.api.serialization.Serializer; import org.apache.seatunnel.api.source.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; @@ -94,6 +95,7 @@ public String getPluginName() { @Override public void prepare(Config pluginConfig) throws PrepareFailException { ReadonlyConfig config = ReadonlyConfig.fromConfig(pluginConfig); + ConfigValidator.of(config).validate(new JdbcSourceFactory().optionRule()); this.jdbcSourceConfig = JdbcSourceConfig.of(config); this.jdbcConnectionProvider = new SimpleJdbcConnectionProvider(jdbcSourceConfig.getJdbcConnectionConfig()); From 0842ec24aae7bc97190f6663408c728f782acda4 Mon Sep 17 00:00:00 2001 From: ic4y <83933160+ic4y@users.noreply.github.com> Date: Tue, 1 Aug 2023 15:08:53 +0800 Subject: [PATCH 36/43] [bugfix][zeta] Fix the issue of two identical IDs appearing when executing seatunnel.sh -l as the job resumes (#5191) --- .../engine/server/master/JobHistoryService.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistoryService.java b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistoryService.java index 12dcae40ca78..dda9a2d0f3fc 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistoryService.java +++ b/seatunnel-engine/seatunnel-engine-server/src/main/java/org/apache/seatunnel/engine/server/master/JobHistoryService.java @@ -42,7 +42,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import java.util.stream.Stream; public class JobHistoryService { @@ -101,10 +103,15 @@ public JobHistoryService( // Gets the status of a running and completed job public String listAllJob() { List status = new ArrayList<>(); + Set runningJonIds = + runningJobMasterMap.values().stream() + .map(master -> master.getJobImmutableInformation().getJobId()) + .collect(Collectors.toSet()); Stream.concat( runningJobMasterMap.values().stream() .map(master -> toJobStateMapper(master, true)), - finishedJobStateImap.values().stream()) + finishedJobStateImap.values().stream() + .filter(jobState -> !runningJonIds.contains(jobState.getJobId()))) .forEach( jobState -> { JobStatusData jobStatusData = From d87f68b534cbc8e56f6970c8d962bf6ebf5e8641 Mon Sep 17 00:00:00 2001 From: monster <60029759+MonsterChenzhuo@users.noreply.github.com> Date: Wed, 2 Aug 2023 10:06:18 +0800 Subject: [PATCH 37/43] [Improve][Docs][Kafka]Reconstruct the kafka connector document (#4778) * [Docs][Connector-V2][Kafka]Reconstruct the kafka connector document --------- Co-authored-by: chenzy15 --- docs/en/connector-v2/sink/Kafka.md | 151 ++++++++++------------- docs/en/connector-v2/source/kafka.md | 172 ++++++++------------------- 2 files changed, 115 insertions(+), 208 deletions(-) diff --git a/docs/en/connector-v2/sink/Kafka.md b/docs/en/connector-v2/sink/Kafka.md index f971e5390b0b..1e258a058adb 100644 --- a/docs/en/connector-v2/sink/Kafka.md +++ b/docs/en/connector-v2/sink/Kafka.md @@ -1,36 +1,52 @@ # Kafka > Kafka sink connector -> - ## Description -Write Rows to a Kafka topic. +## Support Those Engines + +> Spark
+> Flink
+> Seatunnel Zeta
-## Key features +## Key Features - [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [cdc](../../concept/connector-v2-features.md) + +> By default, we will use 2pc to guarantee the message is sent to kafka exactly once. + +## Description + +Write Rows to a Kafka topic. -By default, we will use 2pc to guarantee the message is sent to kafka exactly once. +## Supported DataSource Info -## Options +In order to use the Kafka connector, the following dependencies are required. +They can be downloaded via install-plugin.sh or from the Maven central repository. -| name | type | required | default value | -|----------------------|--------|----------|---------------| -| topic | string | yes | - | -| bootstrap.servers | string | yes | - | -| kafka.config | map | no | - | -| semantics | string | no | NON | -| partition_key_fields | array | no | - | -| partition | int | no | - | -| assign_partitions | array | no | - | -| transaction_prefix | string | no | - | -| format | String | no | json | -| field_delimiter | String | no | , | -| common-options | config | no | - | +| Datasource | Supported Versions | Maven | +|------------|--------------------|-------------------------------------------------------------------------------------------------------------| +| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-kafka) | -### topic [string] +## Sink Options -Kafka Topic. +| Name | Type | Required | Default | Description | +|----------------------|--------|----------|---------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| topic | String | Yes | - | When the table is used as sink, the topic name is the topic to write data to. | +| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | +| kafka.config | Map | No | - | In addition to the above parameters that must be specified by the `Kafka producer` client, the user can also specify multiple non-mandatory parameters for the `producer` client, covering [all the producer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#producerconfigs). | +| semantics | String | No | NON | Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON. | +| partition_key_fields | Array | No | - | Configure which fields are used as the key of the kafka message. | +| partition | Int | No | - | We can specify the partition, all messages will be sent to this partition. | +| assign_partitions | Array | No | - | We can decide which partition to send based on the content of the message. The function of this parameter is to distribute information. | +| transaction_prefix | String | No | - | If semantic is specified as EXACTLY_ONCE, the producer will write all messages in a Kafka transaction,kafka distinguishes different transactions by different transactionId. This parameter is prefix of kafka transactionId, make sure different job use different prefix. | +| format | String | No | json | Data format. The default format is json. Optional text format, canal-json and debezium-json.If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. | +| field_delimiter | String | No | , | Customize the field delimiter for data format. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | + +## Parameter Interpretation + +### Topic Formats Currently two formats are supported: @@ -47,27 +63,13 @@ Currently two formats are supported: If `${name}` is set as the topic. So the first row is sent to Jack topic, and the second row is sent to Mary topic. -### bootstrap.servers [string] - -Kafka Brokers List. - -### kafka.config [kafka producer config] - -In addition to the above parameters that must be specified by the `Kafka producer` client, the user can also specify multiple non-mandatory parameters for the `producer` client, covering [all the producer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#producerconfigs). - -### semantics [string] - -Semantics that can be chosen EXACTLY_ONCE/AT_LEAST_ONCE/NON, default NON. +### Semantics In EXACTLY_ONCE, producer will write all messages in a Kafka transaction that will be committed to Kafka on a checkpoint. - In AT_LEAST_ONCE, producer will wait for all outstanding messages in the Kafka buffers to be acknowledged by the Kafka producer on a checkpoint. - NON does not provide any guarantees: messages may be lost in case of issues on the Kafka broker and messages may be duplicated. -### partition_key_fields [array] - -Configure which fields are used as the key of the kafka message. +### Partition Key Fields For example, if you want to use value of fields from upstream data as key, you can assign field names to this property. @@ -79,55 +81,48 @@ Upstream data is the following: | Mary | 23 | data-example2 | If name is set as the key, then the hash value of the name column will determine which partition the message is sent to. - If not set partition key fields, the null message key will be sent to. - The format of the message key is json, If name is set as the key, for example '{"name":"Jack"}'. - The selected field must be an existing field in the upstream. -### partition [int] - -We can specify the partition, all messages will be sent to this partition. - -### assign_partitions [array] - -We can decide which partition to send based on the content of the message. The function of this parameter is to distribute information. +### Assign Partitions For example, there are five partitions in total, and the assign_partitions field in config is as follows: assign_partitions = ["shoe", "clothing"] - Then the message containing "shoe" will be sent to partition zero ,because "shoe" is subscribed as zero in assign_partitions, and the message containing "clothing" will be sent to partition one.For other messages, the hash algorithm will be used to divide them into the remaining partitions. - This function by `MessageContentPartitioner` class implements `org.apache.kafka.clients.producer.Partitioner` interface.If we need custom partitions, we need to implement this interface as well. -### transaction_prefix [string] - -If semantic is specified as EXACTLY_ONCE, the producer will write all messages in a Kafka transaction. -Kafka distinguishes different transactions by different transactionId. This parameter is prefix of kafka transactionId, make sure different job use different prefix. - -### format +## Task Example -Data format. The default format is json. Optional text format, canal-json and debezium-json. -If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option. -If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details. -If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. +### Simple: -### field_delimiter +> This example defines a SeaTunnel synchronization task that automatically generates data through FakeSource and sends it to Kafka 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 topic is test_topic will also be 16 rows of data in the topic. 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. -Customize the field delimiter for data format. - -### common options [config] - -Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details. +```hocon +# Defining the runtime environment +env { + # You can set flink configuration here + execution.parallelism = 1 + job.mode = "BATCH" +} -## Examples +source { + FakeSource { + parallelism = 1 + result_table_name = "fake" + row.num = 16 + schema = { + fields { + name = "string" + age = "int" + } + } + } +} -```hocon sink { - kafka { - topic = "seatunnel" + topic = "test_topic" bootstrap.servers = "localhost:9092" partition = 3 format = json @@ -139,7 +134,6 @@ sink { buffer.memory = 33554432 } } - } ``` @@ -162,7 +156,6 @@ sink { sasl.jaas.config="org.apache.kafka.common.security.scram.ScramLoginModule required \nusername=${username}\npassword=${password};" } } - } ``` @@ -199,22 +192,6 @@ sink { sasl.client.callback.handler.class="software.amazon.msk.auth.iam.IAMClientCallbackHandler" } } - } ``` -## Changelog - -### 2.3.0-beta 2022-10-20 - -- Add Kafka Sink Connector - -### next version - -- [Improve] Support to specify multiple partition keys [3230](https://github.com/apache/incubator-seatunnel/pull/3230) -- [Improve] Add text format for kafka sink connector [3711](https://github.com/apache/incubator-seatunnel/pull/3711) -- [Improve] Support extract topic from SeaTunnelRow fields [3742](https://github.com/apache/incubator-seatunnel/pull/3742) -- [Improve] Change Connector Custom Config Prefix To Map [3719](https://github.com/apache/incubator-seatunnel/pull/3719) -- [Improve] Support read canal format message [3950](https://github.com/apache/incubator-seatunnel/pull/3950) -- [Improve] Support read debezium format message [3981](https://github.com/apache/incubator-seatunnel/pull/3981) - diff --git a/docs/en/connector-v2/source/kafka.md b/docs/en/connector-v2/source/kafka.md index 2ed6ec6f12e4..16b9c5420b3f 100644 --- a/docs/en/connector-v2/source/kafka.md +++ b/docs/en/connector-v2/source/kafka.md @@ -2,11 +2,13 @@ > Kafka source connector -## Description +## Support Those Engines -Source connector for Apache Kafka. +> Spark
+> Flink
+> Seatunnel Zeta
-## Key features +## Key Features - [x] [batch](../../concept/connector-v2-features.md) - [x] [stream](../../concept/connector-v2-features.md) @@ -15,111 +17,54 @@ Source connector for Apache Kafka. - [x] [parallelism](../../concept/connector-v2-features.md) - [ ] [support user-defined split](../../concept/connector-v2-features.md) -## Options - -| name | type | required | default value | -|-------------------------------------|---------|----------|--------------------------| -| topic | String | yes | - | -| bootstrap.servers | String | yes | - | -| pattern | Boolean | no | false | -| consumer.group | String | no | SeaTunnel-Consumer-Group | -| commit_on_checkpoint | Boolean | no | true | -| kafka.config | Map | no | - | -| common-options | config | no | - | -| schema | | no | - | -| format | String | no | json | -| format_error_handle_way | String | no | fail | -| field_delimiter | String | no | , | -| start_mode | String | no | group_offsets | -| start_mode.offsets | | no | | -| start_mode.timestamp | Long | no | | -| partition-discovery.interval-millis | long | no | -1 | - -### topic [string] - -`Kafka topic` name. If there are multiple `topics`, use `,` to split, for example: `"tpc1,tpc2"`. - -### bootstrap.servers [string] - -`Kafka` cluster address, separated by `","`. - -### pattern [boolean] - -If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. - -### consumer.group [string] - -`Kafka consumer group id`, used to distinguish different consumer groups. - -### commit_on_checkpoint [boolean] - -If true the consumer's offset will be periodically committed in the background. - -## partition-discovery.interval-millis [long] - -The interval for dynamically discovering topics and partitions. - -### kafka.config [map] - -In addition to the above necessary parameters that must be specified by the `Kafka consumer` client, users can also specify multiple `consumer` client non-mandatory parameters, covering [all consumer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#consumerconfigs). - -### common-options [config] - -Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. - -### schema - -The structure of the data, including field names and field types. - -## format - -Data format. The default format is json. Optional text format, canal-json and debezium-json. -If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option. -If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details. -If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. - -## format_error_handle_way - -The processing method of data format error. The default value is fail, and the optional value is (fail, skip). -When fail is selected, data format error will block and an exception will be thrown. -When skip is selected, data format error will skip this line data. - -## field_delimiter - -Customize the field delimiter for data format. - -## start_mode - -The initial consumption pattern of consumers,there are several types: -[earliest],[group_offsets],[latest],[specific_offsets],[timestamp] - -## start_mode.timestamp - -The time required for consumption mode to be "timestamp". - -## start_mode.offsets - -The offset required for consumption mode to be specific_offsets. - -for example: +## Description -```hocon -start_mode.offsets = { - info-0 = 70 - info-1 = 10 - info-2 = 10 - } -``` +Source connector for Apache Kafka. -## Example +## Supported DataSource Info + +In order to use the Kafka connector, the following dependencies are required. +They can be downloaded via install-plugin.sh or from the Maven central repository. + +| Datasource | Supported Versions | Maven | +|------------|--------------------|-------------------------------------------------------------------------------------------------------------| +| Kafka | Universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-kafka) | + +## Source Options + +| Name | Type | Required | Default | Description | +|-------------------------------------|-----------------------------------------------------------------------------|----------|--------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| topic | String | Yes | - | Topic name(s) to read data from when the table is used as source. It also supports topic list for source by separating topic by comma like 'topic-1,topic-2'. | +| bootstrap.servers | String | Yes | - | Comma separated list of Kafka brokers. | +| pattern | Boolean | No | false | If `pattern` is set to `true`,the regular expression for a pattern of topic names to read from. All topics in clients with names that match the specified regular expression will be subscribed by the consumer. | +| consumer.group | String | No | SeaTunnel-Consumer-Group | `Kafka consumer group id`, used to distinguish different consumer groups. | +| commit_on_checkpoint | Boolean | No | true | If true the consumer's offset will be periodically committed in the background. | +| kafka.config | Map | No | - | In addition to the above necessary parameters that must be specified by the `Kafka consumer` client, users can also specify multiple `consumer` client non-mandatory parameters, covering [all consumer parameters specified in the official Kafka document](https://kafka.apache.org/documentation.html#consumerconfigs). | +| schema | Config | No | - | The structure of the data, including field names and field types. | +| format | String | No | json | Data format. The default format is json. Optional text format, canal-json and debezium-json.If you use json or text format. The default field separator is ", ". If you customize the delimiter, add the "field_delimiter" option.If you use canal format, please refer to [canal-json](../formats/canal-json.md) for details.If you use debezium format, please refer to [debezium-json](../formats/debezium-json.md) for details. | +| format_error_handle_way | String | No | fail | The processing method of data format error. The default value is fail, and the optional value is (fail, skip). When fail is selected, data format error will block and an exception will be thrown. When skip is selected, data format error will skip this line data. | +| field_delimiter | String | No | , | Customize the field delimiter for data format. | +| start_mode | StartMode[earliest],[group_offsets],[latest],[specific_offsets],[timestamp] | No | group_offsets | The initial consumption pattern of consumers. | +| start_mode.offsets | Config | No | - | The offset required for consumption mode to be specific_offsets. | +| start_mode.timestamp | Long | No | - | The time required for consumption mode to be "timestamp". | +| partition-discovery.interval-millis | Long | No | -1 | The interval for dynamically discovering topics and partitions. | +| common-options | | No | - | Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details | + +## Task Example ### Simple +> This example reads the data of kafka's topic_1, topic_2, topic_3 and prints it to the client.And if you have not yet installed and deployed SeaTunnel, you need to follow the instructions in Install SeaTunnel to install and deploy SeaTunnel. 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. + ```hocon +# Defining the runtime environment +env { + # You can set flink configuration here + execution.parallelism = 2 + job.mode = "BATCH" +} source { - Kafka { - result_table_name = "kafka_name" schema = { fields { name = "string" @@ -136,8 +81,10 @@ source { auto.offset.reset = "earliest" enable.auto.commit = "false" } - } - + } +} +sink { + Console {} } ``` @@ -145,14 +92,12 @@ source { ```hocon source { - Kafka { topic = ".*seatunnel*." pattern = "true" bootstrap.servers = "localhost:9092" consumer.group = "seatunnel_group" } - } ``` @@ -169,7 +114,7 @@ source { kafka.config = { security.protocol=SASL_SSL sasl.mechanism=SCRAM-SHA-512 - sasl.jaas.config="org.apache.kafka.common.security.scram.ScramLoginModule required \nusername=${username}\npassword=${password};" + sasl.jaas.config="org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";" #security.protocol=SASL_SSL #sasl.mechanism=AWS_MSK_IAM #sasl.jaas.config="software.amazon.msk.auth.iam.IAMLoginModule required;" @@ -205,7 +150,7 @@ source { kafka.config = { #security.protocol=SASL_SSL #sasl.mechanism=SCRAM-SHA-512 - #sasl.jaas.config="org.apache.kafka.common.security.scram.ScramLoginModule required \nusername=${username}\npassword=${password};" + #sasl.jaas.config="org.apache.kafka.common.security.scram.ScramLoginModule required username=\"username\" password=\"password\";" security.protocol=SASL_SSL sasl.mechanism=AWS_MSK_IAM sasl.jaas.config="software.amazon.msk.auth.iam.IAMLoginModule required;" @@ -215,18 +160,3 @@ source { } ``` -## Changelog - -### 2.3.0-beta 2022-10-20 - -- Add Kafka Source Connector - -### Next Version - -- [Improve] Support setting read starting offset or time at startup config ([3157](https://github.com/apache/incubator-seatunnel/pull/3157)) -- [Improve] Support for dynamic discover topic & partition in streaming mode ([3125](https://github.com/apache/incubator-seatunnel/pull/3125)) -- [Improve] Change Connector Custom Config Prefix To Map [3719](https://github.com/apache/incubator-seatunnel/pull/3719) -- [Bug] Fixed the problem that parsing the offset format failed when the startup mode was offset([3810](https://github.com/apache/incubator-seatunnel/pull/3810)) -- [Improve] Support read canal format message [3950](https://github.com/apache/incubator-seatunnel/pull/3950) -- [Improve] Support read debezium format message [3981](https://github.com/apache/incubator-seatunnel/pull/3981) - From e717d80166bdc3e814208f49ffb2421d33979a5b Mon Sep 17 00:00:00 2001 From: lightzhao <40714172+lightzhao@users.noreply.github.com> Date: Wed, 2 Aug 2023 16:34:18 +0800 Subject: [PATCH 38/43] [Bug][Improve][LocalFileSink]Fix LocalFile Sink file_format_type. (#5118) --- docs/en/connector-v2/sink/FtpFile.md | 18 +++++++------- docs/en/connector-v2/sink/HdfsFile.md | 12 +++++----- docs/en/connector-v2/sink/LocalFile.md | 24 +++++++++---------- docs/en/connector-v2/sink/OssFile.md | 10 ++++---- docs/en/connector-v2/sink/OssJindoFile.md | 10 ++++---- docs/en/connector-v2/sink/S3-Redshift.md | 2 +- docs/en/connector-v2/sink/S3File.md | 10 ++++---- docs/en/connector-v2/sink/SftpFile.md | 10 ++++---- docs/en/connector-v2/source/FtpFile.md | 2 +- docs/en/connector-v2/source/HdfsFile.md | 2 +- docs/en/connector-v2/source/LocalFile.md | 2 +- docs/en/connector-v2/source/OssFile.md | 2 +- docs/en/connector-v2/source/OssJindoFile.md | 2 +- docs/en/connector-v2/source/S3File.md | 2 +- docs/en/connector-v2/source/SftpFile.md | 2 +- .../src/test/resources/conf/option-test.conf | 2 +- ...ke_to_localfile_two_pipeline_template.conf | 2 +- .../resources/batch_fakesource_to_file.conf | 2 +- .../batch_fakesource_to_file_complex.conf | 2 +- .../src/test/resources/client_test.conf | 2 +- .../resources/batch_fakesource_to_file.conf | 2 +- .../batch_fakesource_to_file_complex.conf | 2 +- .../resources/stream_fakesource_to_file.conf | 2 +- .../stream_fakesource_to_file_savepoint.conf | 2 +- 24 files changed, 64 insertions(+), 64 deletions(-) diff --git a/docs/en/connector-v2/sink/FtpFile.md b/docs/en/connector-v2/sink/FtpFile.md index b92bcd7fcc37..8b3214e44b3c 100644 --- a/docs/en/connector-v2/sink/FtpFile.md +++ b/docs/en/connector-v2/sink/FtpFile.md @@ -40,9 +40,9 @@ By default, we use 2PC commit to ensure `exactly-once` | custom_filename | boolean | no | false | Whether you need custom the filename | | file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | | filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | -| file_format | string | no | "csv" | | -| field_delimiter | string | no | '\001' | Only used when file_format is text | -| row_delimiter | string | no | "\n" | Only used when file_format is text | +| file_format_type | string | no | "csv" | | +| field_delimiter | string | no | '\001' | Only used when file_format_type is text | +| row_delimiter | string | no | "\n" | Only used when file_format_type is text | | have_partition | boolean | no | false | Whether you need processing partitions. | | partition_by | array | no | - | Only used then have_partition is true | | partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | @@ -52,8 +52,8 @@ By default, we use 2PC commit to ensure `exactly-once` | batch_size | int | no | 1000000 | | | compress_codec | string | no | none | | | common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | ### host [string] @@ -103,13 +103,13 @@ When the format in the `file_name_expression` parameter is `xxxx-${now}` , `file | m | Minute in hour | | s | Second in minute | -### file_format [string] +### file_format_type [string] We supported as the following file types: `text` `json` `csv` `orc` `parquet` `excel` -Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. +Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. ### field_delimiter [string] @@ -198,7 +198,7 @@ FtpFile { username = "username" password = "password" path = "/data/ftp" - file_format = "text" + file_format_type = "text" field_delimiter = "\t" row_delimiter = "\n" sink_columns = ["name","age"] @@ -216,7 +216,7 @@ FtpFile { username = "username" password = "password" path = "/data/ftp" - file_format = "text" + file_format_type = "text" field_delimiter = "\t" row_delimiter = "\n" have_partition = true diff --git a/docs/en/connector-v2/sink/HdfsFile.md b/docs/en/connector-v2/sink/HdfsFile.md index 1e094a5e573c..34ce19714b4d 100644 --- a/docs/en/connector-v2/sink/HdfsFile.md +++ b/docs/en/connector-v2/sink/HdfsFile.md @@ -41,8 +41,8 @@ By default, we use 2PC commit to ensure `exactly-once` | file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | | filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | | file_format_type | string | no | "csv" | | -| field_delimiter | string | no | '\001' | Only used when file_format is text | -| row_delimiter | string | no | "\n" | Only used when file_format is text | +| field_delimiter | string | no | '\001' | Only used when file_format_type is text | +| row_delimiter | string | no | "\n" | Only used when file_format_type is text | | have_partition | boolean | no | false | Whether you need processing partitions. | | partition_by | array | no | - | Only used then have_partition is true | | partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | @@ -55,8 +55,8 @@ By default, we use 2PC commit to ensure `exactly-once` | kerberos_keytab_path | string | no | - | | | compress_codec | string | no | none | | | common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | ### fs.defaultFS [string] @@ -104,7 +104,7 @@ We supported as the following file types: `text` `json` `csv` `orc` `parquet` `excel` -Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. +Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. ### field_delimiter [string] @@ -198,7 +198,7 @@ For orc file format simple config HdfsFile { fs.defaultFS = "hdfs://hadoopcluster" path = "/tmp/hive/warehouse/test2" - file_format = "orc" + file_format_type = "orc" } ``` diff --git a/docs/en/connector-v2/sink/LocalFile.md b/docs/en/connector-v2/sink/LocalFile.md index fb008e909a9c..8e2c1526e907 100644 --- a/docs/en/connector-v2/sink/LocalFile.md +++ b/docs/en/connector-v2/sink/LocalFile.md @@ -20,7 +20,7 @@ If you use SeaTunnel Engine, It automatically integrated the hadoop jar when you By default, we use 2PC commit to ensure `exactly-once` -- [x] file format +- [x] file format type - [x] text - [x] csv - [x] parquet @@ -36,9 +36,9 @@ By default, we use 2PC commit to ensure `exactly-once` | custom_filename | boolean | no | false | Whether you need custom the filename | | file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | | filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | -| file_format | string | no | "csv" | | -| field_delimiter | string | no | '\001' | Only used when file_format is text | -| row_delimiter | string | no | "\n" | Only used when file_format is text | +| file_format_type | string | no | "csv" | | +| field_delimiter | string | no | '\001' | Only used when file_format_type is text | +| row_delimiter | string | no | "\n" | Only used when file_format_type is text | | have_partition | boolean | no | false | Whether you need processing partitions. | | partition_by | array | no | - | Only used then have_partition is true | | partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | @@ -48,8 +48,8 @@ By default, we use 2PC commit to ensure `exactly-once` | batch_size | int | no | 1000000 | | | compress_codec | string | no | none | | | common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | ### path [string] @@ -83,13 +83,13 @@ When the format in the `file_name_expression` parameter is `xxxx-${now}` , `file | m | Minute in hour | | s | Second in minute | -### file_format [string] +### file_format_type [string] We supported as the following file types: `text` `json` `csv` `orc` `parquet` `excel` -Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. +Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. ### field_delimiter [string] @@ -174,7 +174,7 @@ For orc file format simple config LocalFile { path = "/tmp/hive/warehouse/test2" - file_format = "orc" + file_format_type = "orc" } ``` @@ -185,7 +185,7 @@ For parquet file format with `sink_columns` LocalFile { path = "/tmp/hive/warehouse/test2" - file_format = "parquet" + file_format_type = "parquet" sink_columns = ["name","age"] } @@ -197,7 +197,7 @@ For text file format with `have_partition` and `custom_filename` and `sink_colum LocalFile { path = "/tmp/hive/warehouse/test2" - file_format = "text" + file_format_type = "text" field_delimiter = "\t" row_delimiter = "\n" have_partition = true @@ -224,7 +224,7 @@ LocalFile { partition_dir_expression="${k0}=${v0}" is_partition_field_write_in_file=true file_name_expression="${transactionId}_${now}" - file_format="excel" + file_format_type="excel" filename_time_format="yyyy.MM.dd" is_enable_transaction=true } diff --git a/docs/en/connector-v2/sink/OssFile.md b/docs/en/connector-v2/sink/OssFile.md index d40cf4bf958c..a3095ecfd1a4 100644 --- a/docs/en/connector-v2/sink/OssFile.md +++ b/docs/en/connector-v2/sink/OssFile.md @@ -44,8 +44,8 @@ By default, we use 2PC commit to ensure `exactly-once` | file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | | filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | | file_format_type | string | no | "csv" | | -| field_delimiter | string | no | '\001' | Only used when file_format is text | -| row_delimiter | string | no | "\n" | Only used when file_format is text | +| field_delimiter | string | no | '\001' | Only used when file_format_type is text | +| row_delimiter | string | no | "\n" | Only used when file_format_type is text | | have_partition | boolean | no | false | Whether you need processing partitions. | | partition_by | array | no | - | Only used then have_partition is true | | partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | @@ -55,8 +55,8 @@ By default, we use 2PC commit to ensure `exactly-once` | batch_size | int | no | 1000000 | | | compress_codec | string | no | none | | | common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | ### path [string] @@ -112,7 +112,7 @@ We supported as the following file types: `text` `json` `csv` `orc` `parquet` `excel` -Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. +Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. ### field_delimiter [string] diff --git a/docs/en/connector-v2/sink/OssJindoFile.md b/docs/en/connector-v2/sink/OssJindoFile.md index 02547f3aa6a7..1d098da009c4 100644 --- a/docs/en/connector-v2/sink/OssJindoFile.md +++ b/docs/en/connector-v2/sink/OssJindoFile.md @@ -44,8 +44,8 @@ By default, we use 2PC commit to ensure `exactly-once` | file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | | filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | | file_format_type | string | no | "csv" | | -| field_delimiter | string | no | '\001' | Only used when file_format is text | -| row_delimiter | string | no | "\n" | Only used when file_format is text | +| field_delimiter | string | no | '\001' | Only used when file_format_type is text | +| row_delimiter | string | no | "\n" | Only used when file_format_type is text | | have_partition | boolean | no | false | Whether you need processing partitions. | | partition_by | array | no | - | Only used then have_partition is true | | partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | @@ -55,8 +55,8 @@ By default, we use 2PC commit to ensure `exactly-once` | batch_size | int | no | 1000000 | | | compress_codec | string | no | none | | | common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | ### path [string] @@ -112,7 +112,7 @@ We supported as the following file types: `text` `json` `csv` `orc` `parquet` `excel` -Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. +Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. ### field_delimiter [string] diff --git a/docs/en/connector-v2/sink/S3-Redshift.md b/docs/en/connector-v2/sink/S3-Redshift.md index 978ffc7c94fc..2e02e2f446a7 100644 --- a/docs/en/connector-v2/sink/S3-Redshift.md +++ b/docs/en/connector-v2/sink/S3-Redshift.md @@ -124,7 +124,7 @@ We supported as the following file types: `text` `csv` `parquet` `orc` `json` -Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. +Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. ### filename_time_format [string] diff --git a/docs/en/connector-v2/sink/S3File.md b/docs/en/connector-v2/sink/S3File.md index dcaee7338fe4..7841afdf04e3 100644 --- a/docs/en/connector-v2/sink/S3File.md +++ b/docs/en/connector-v2/sink/S3File.md @@ -44,8 +44,8 @@ By default, we use 2PC commit to ensure `exactly-once` | file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | | filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | | file_format_type | string | no | "csv" | | -| field_delimiter | string | no | '\001' | Only used when file_format is text | -| row_delimiter | string | no | "\n" | Only used when file_format is text | +| field_delimiter | string | no | '\001' | Only used when file_format_type is text | +| row_delimiter | string | no | "\n" | Only used when file_format_type is text | | have_partition | boolean | no | false | Whether you need processing partitions. | | partition_by | array | no | - | Only used then have_partition is true | | partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | @@ -55,8 +55,8 @@ By default, we use 2PC commit to ensure `exactly-once` | batch_size | int | no | 1000000 | | | compress_codec | string | no | none | | | common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | ### path [string] @@ -129,7 +129,7 @@ We supported as the following file types: `text` `json` `csv` `orc` `parquet` `excel` -Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. +Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. ### field_delimiter [string] diff --git a/docs/en/connector-v2/sink/SftpFile.md b/docs/en/connector-v2/sink/SftpFile.md index 79643b8c8aa2..b6460f39e398 100644 --- a/docs/en/connector-v2/sink/SftpFile.md +++ b/docs/en/connector-v2/sink/SftpFile.md @@ -41,8 +41,8 @@ By default, we use 2PC commit to ensure `exactly-once` | file_name_expression | string | no | "${transactionId}" | Only used when custom_filename is true | | filename_time_format | string | no | "yyyy.MM.dd" | Only used when custom_filename is true | | file_format_type | string | no | "csv" | | -| field_delimiter | string | no | '\001' | Only used when file_format is text | -| row_delimiter | string | no | "\n" | Only used when file_format is text | +| field_delimiter | string | no | '\001' | Only used when file_format_type is text | +| row_delimiter | string | no | "\n" | Only used when file_format_type is text | | have_partition | boolean | no | false | Whether you need processing partitions. | | partition_by | array | no | - | Only used then have_partition is true | | partition_dir_expression | string | no | "${k0}=${v0}/${k1}=${v1}/.../${kn}=${vn}/" | Only used then have_partition is true | @@ -52,8 +52,8 @@ By default, we use 2PC commit to ensure `exactly-once` | batch_size | int | no | 1000000 | | | compress_codec | string | no | none | | | common-options | object | no | - | | -| max_rows_in_memory | int | no | - | Only used when file_format is excel. | -| sheet_name | string | no | Sheet${Random number} | Only used when file_format is excel. | +| max_rows_in_memory | int | no | - | Only used when file_format_type is excel. | +| sheet_name | string | no | Sheet${Random number} | Only used when file_format_type is excel. | ### host [string] @@ -109,7 +109,7 @@ We supported as the following file types: `text` `json` `csv` `orc` `parquet` `excel` -Please note that, The final file name will end with the file_format's suffix, the suffix of the text file is `txt`. +Please note that, The final file name will end with the file_format_type's suffix, the suffix of the text file is `txt`. ### field_delimiter [string] diff --git a/docs/en/connector-v2/source/FtpFile.md b/docs/en/connector-v2/source/FtpFile.md index 6737511e63db..c692a7483a6d 100644 --- a/docs/en/connector-v2/source/FtpFile.md +++ b/docs/en/connector-v2/source/FtpFile.md @@ -226,7 +226,7 @@ Source plugin common parameters, please refer to [Source Common Options](common- ### sheet_name [string] -Reader the sheet of the workbook,Only used when file_format is excel. +Reader the sheet of the workbook,Only used when file_format_type is excel. ## Example diff --git a/docs/en/connector-v2/source/HdfsFile.md b/docs/en/connector-v2/source/HdfsFile.md index 1d285c539a3f..f479e40a2bc2 100644 --- a/docs/en/connector-v2/source/HdfsFile.md +++ b/docs/en/connector-v2/source/HdfsFile.md @@ -244,7 +244,7 @@ Source plugin common parameters, please refer to [Source Common Options](common- ### sheet_name [string] -Reader the sheet of the workbook,Only used when file_format is excel. +Reader the sheet of the workbook,Only used when file_format_type is excel. ### file_filter_pattern [string] diff --git a/docs/en/connector-v2/source/LocalFile.md b/docs/en/connector-v2/source/LocalFile.md index 7d80a4d2b7a3..80adfa6d9ad9 100644 --- a/docs/en/connector-v2/source/LocalFile.md +++ b/docs/en/connector-v2/source/LocalFile.md @@ -224,7 +224,7 @@ Source plugin common parameters, please refer to [Source Common Options](common- ### sheet_name [string] -Reader the sheet of the workbook,Only used when file_format is excel. +Reader the sheet of the workbook,Only used when file_format_type is excel. ### file_filter_pattern [string] diff --git a/docs/en/connector-v2/source/OssFile.md b/docs/en/connector-v2/source/OssFile.md index 12f2141cd6ed..7c992581f5a6 100644 --- a/docs/en/connector-v2/source/OssFile.md +++ b/docs/en/connector-v2/source/OssFile.md @@ -247,7 +247,7 @@ Source plugin common parameters, please refer to [Source Common Options](common- ### sheet_name [string] -Reader the sheet of the workbook,Only used when file_format is excel. +Reader the sheet of the workbook,Only used when file_format_type is excel. ## Example diff --git a/docs/en/connector-v2/source/OssJindoFile.md b/docs/en/connector-v2/source/OssJindoFile.md index 913d277683e4..f77c4a4543a9 100644 --- a/docs/en/connector-v2/source/OssJindoFile.md +++ b/docs/en/connector-v2/source/OssJindoFile.md @@ -247,7 +247,7 @@ Source plugin common parameters, please refer to [Source Common Options](common- ### sheet_name [string] -Reader the sheet of the workbook,Only used when file_format is excel. +Reader the sheet of the workbook,Only used when file_format_type is excel. ### file_filter_pattern [string] diff --git a/docs/en/connector-v2/source/S3File.md b/docs/en/connector-v2/source/S3File.md index 79a89be1c27f..f7ad1cc8bd0f 100644 --- a/docs/en/connector-v2/source/S3File.md +++ b/docs/en/connector-v2/source/S3File.md @@ -264,7 +264,7 @@ Source plugin common parameters, please refer to [Source Common Options](common- ### sheet_name [string] -Reader the sheet of the workbook,Only used when file_format is excel. +Reader the sheet of the workbook,Only used when file_format_type is excel. ## Example diff --git a/docs/en/connector-v2/source/SftpFile.md b/docs/en/connector-v2/source/SftpFile.md index 22047d481ed6..184a587a9286 100644 --- a/docs/en/connector-v2/source/SftpFile.md +++ b/docs/en/connector-v2/source/SftpFile.md @@ -225,7 +225,7 @@ Source plugin common parameters, please refer to [Source Common Options](common- ### sheet_name [string] -Reader the sheet of the workbook,Only used when file_format is excel. +Reader the sheet of the workbook,Only used when file_format_type is excel. ### file_filter_pattern [string] diff --git a/seatunnel-api/src/test/resources/conf/option-test.conf b/seatunnel-api/src/test/resources/conf/option-test.conf index 4f20d493d4c0..9461e5298b98 100644 --- a/seatunnel-api/src/test/resources/conf/option-test.conf +++ b/seatunnel-api/src/test/resources/conf/option-test.conf @@ -101,7 +101,7 @@ sink { partition_dir_expression = "${k0}=${v0}" is_partition_field_write_in_file = true file_name_expression = "${transactionId}_${now}" - file_format = "text" + file_format_type = "text" sink_columns = ["name","age"] } } \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/cluster_batch_fake_to_localfile_two_pipeline_template.conf b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/cluster_batch_fake_to_localfile_two_pipeline_template.conf index 7e01c01c984d..e94f7ae672e8 100644 --- a/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/cluster_batch_fake_to_localfile_two_pipeline_template.conf +++ b/seatunnel-e2e/seatunnel-engine-e2e/connector-seatunnel-e2e-base/src/test/resources/cluster_batch_fake_to_localfile_two_pipeline_template.conf @@ -134,7 +134,7 @@ sink { field_delimiter = "\t" row_delimiter = "\n" file_name_expression = "${transactionId}_${now}" - file_format = "text" + file_format_type = "text" filename_time_format = "yyyy.MM.dd" is_enable_transaction = true save_mode = "error" diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fakesource_to_file.conf b/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fakesource_to_file.conf index 4f5cffa4a929..181a9fc1ad71 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fakesource_to_file.conf +++ b/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fakesource_to_file.conf @@ -52,7 +52,7 @@ sink { partition_dir_expression = "${k0}=${v0}" is_partition_field_write_in_file = true file_name_expression = "${transactionId}_${now}" - file_format = "text" + file_format_type = "text" sink_columns = ["name", "age"] filename_time_format = "yyyy.MM.dd" is_enable_transaction = true diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fakesource_to_file_complex.conf b/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fakesource_to_file_complex.conf index c687fc1cf247..3a44886274ec 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fakesource_to_file_complex.conf +++ b/seatunnel-engine/seatunnel-engine-client/src/test/resources/batch_fakesource_to_file_complex.conf @@ -63,7 +63,7 @@ sink { partition_dir_expression = "${k0}=${v0}" is_partition_field_write_in_file = true file_name_expression = "${transactionId}_${now}" - file_format = "text" + file_format_type = "text" sink_columns = ["name", "age"] filename_time_format = "yyyy.MM.dd" is_enable_transaction = true diff --git a/seatunnel-engine/seatunnel-engine-client/src/test/resources/client_test.conf b/seatunnel-engine/seatunnel-engine-client/src/test/resources/client_test.conf index 92e159c2a243..a4404b9f918d 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/test/resources/client_test.conf +++ b/seatunnel-engine/seatunnel-engine-client/src/test/resources/client_test.conf @@ -63,7 +63,7 @@ sink { partition_dir_expression="${k0}=${v0}" is_partition_field_write_in_file=true file_name_expression="${transactionId}_${now}" - file_format="text" + file_format_type="text" sink_columns=["name","age"] filename_time_format="yyyy.MM.dd" is_enable_transaction=true diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/resources/batch_fakesource_to_file.conf b/seatunnel-engine/seatunnel-engine-server/src/test/resources/batch_fakesource_to_file.conf index 3e71d5dfb17d..24339945e79d 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/resources/batch_fakesource_to_file.conf +++ b/seatunnel-engine/seatunnel-engine-server/src/test/resources/batch_fakesource_to_file.conf @@ -52,7 +52,7 @@ sink { partition_dir_expression="${k0}=${v0}" is_partition_field_write_in_file=true file_name_expression="${transactionId}_${now}" - file_format="text" + file_format_type="text" sink_columns=["name","age"] filename_time_format="yyyy.MM.dd" is_enable_transaction=true diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/resources/batch_fakesource_to_file_complex.conf b/seatunnel-engine/seatunnel-engine-server/src/test/resources/batch_fakesource_to_file_complex.conf index 4e345cf96774..e3e0e00d9b05 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/resources/batch_fakesource_to_file_complex.conf +++ b/seatunnel-engine/seatunnel-engine-server/src/test/resources/batch_fakesource_to_file_complex.conf @@ -63,7 +63,7 @@ sink { partition_dir_expression="${k0}=${v0}" is_partition_field_write_in_file=true file_name_expression="${transactionId}_${now}" - file_format="text" + file_format_type="text" sink_columns=["name","age"] filename_time_format="yyyy.MM.dd" is_enable_transaction=true diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file.conf b/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file.conf index 6f86a81d464a..d2299ecfd2e2 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file.conf +++ b/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file.conf @@ -52,7 +52,7 @@ sink { partition_dir_expression="${k0}=${v0}" is_partition_field_write_in_file=true file_name_expression="${transactionId}_${now}" - file_format="text" + file_format_type="text" sink_columns=["name","age"] filename_time_format="yyyy.MM.dd" is_enable_transaction=true diff --git a/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file_savepoint.conf b/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file_savepoint.conf index ced94b31e5ff..275750298099 100644 --- a/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file_savepoint.conf +++ b/seatunnel-engine/seatunnel-engine-server/src/test/resources/stream_fakesource_to_file_savepoint.conf @@ -55,7 +55,7 @@ sink { partition_dir_expression="${k0}=${v0}" is_partition_field_write_in_file=true file_name_expression="${transactionId}_${now}" - file_format="text" + file_format_type="text" sink_columns=["name","age"] filename_time_format="yyyy.MM.dd" is_enable_transaction=true From 9af696a1dd1cff2e9b3773f38f7c81501abcfe16 Mon Sep 17 00:00:00 2001 From: kun <66303359+Lifu12@users.noreply.github.com> Date: Wed, 2 Aug 2023 16:55:08 +0800 Subject: [PATCH 39/43] [Bug] [connector-v2] PostgreSQL versions below 9.5 are compatible use cdc sync problem (#5120) --- docs/en/connector-v2/sink/Jdbc.md | 22 ++++++++++ .../dialect/psql/PostgresDialectFactory.java | 12 ++++++ .../dialect/psqllow/PostgresLowDialect.java | 30 ++++++++++++++ .../dialect/PostgresDialectFactoryTest.java | 40 +++++++++++++++++++ 4 files changed, 104 insertions(+) create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psqllow/PostgresLowDialect.java create mode 100644 seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/PostgresDialectFactoryTest.java diff --git a/docs/en/connector-v2/sink/Jdbc.md b/docs/en/connector-v2/sink/Jdbc.md index f128f6b4b218..9d68278cf51e 100644 --- a/docs/en/connector-v2/sink/Jdbc.md +++ b/docs/en/connector-v2/sink/Jdbc.md @@ -74,6 +74,8 @@ Use this sql write upstream input datas to database. e.g `INSERT ...` The compatible mode of database, required when the database supports multiple compatible modes. For example, when using OceanBase database, you need to set it to 'mysql' or 'oracle'. +Postgres 9.5 version or below,please set it to `postgresLow` to support cdc + ### database [string] Use this `database` and `table-name` auto-generate sql and receive upstream input datas write to database. @@ -226,6 +228,26 @@ sink { } ``` +Postgresql 9.5 version below support CDC(Change data capture) event + +``` +sink { + jdbc { + url = "jdbc:postgresql://localhost:5432" + driver = "org.postgresql.Driver" + user = "root" + password = "123456" + compatible_mode="postgresLow" + database = "sink_database" + table = "sink_table" + support_upsert_by_query_primary_key_exist = true + generate_sink_sql = true + primary_keys = ["key1", "key2", ...] + } +} + +``` + ## Changelog ### 2.2.0-beta 2022-09-26 diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresDialectFactory.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresDialectFactory.java index 963f7385e231..857c85290df6 100644 --- a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresDialectFactory.java +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psql/PostgresDialectFactory.java @@ -19,9 +19,12 @@ import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialect; import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.JdbcDialectFactory; +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.psqllow.PostgresLowDialect; import com.google.auto.service.AutoService; +import javax.annotation.Nonnull; + @AutoService(JdbcDialectFactory.class) public class PostgresDialectFactory implements JdbcDialectFactory { @Override @@ -31,6 +34,15 @@ public boolean acceptsURL(String url) { @Override public JdbcDialect create() { + throw new UnsupportedOperationException( + "Can't create JdbcDialect without compatible mode for Postgres"); + } + + @Override + public JdbcDialect create(@Nonnull String compatibleMode) { + if ("postgresLow".equalsIgnoreCase(compatibleMode)) { + return new PostgresLowDialect(); + } return new PostgresDialect(); } } diff --git a/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psqllow/PostgresLowDialect.java b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psqllow/PostgresLowDialect.java new file mode 100644 index 000000000000..e367207ffa20 --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/main/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/psqllow/PostgresLowDialect.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.psqllow; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.psql.PostgresDialect; + +import java.util.Optional; + +public class PostgresLowDialect extends PostgresDialect { + @Override + public Optional getUpsertStatement( + String database, String tableName, String[] fieldNames, String[] uniqueKeyFields) { + return Optional.empty(); + } +} diff --git a/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/PostgresDialectFactoryTest.java b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/PostgresDialectFactoryTest.java new file mode 100644 index 000000000000..79b1f11ac93b --- /dev/null +++ b/seatunnel-connectors-v2/connector-jdbc/src/test/java/org/apache/seatunnel/connectors/seatunnel/jdbc/internal/dialect/PostgresDialectFactoryTest.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect; + +import org.apache.seatunnel.connectors.seatunnel.jdbc.internal.dialect.psql.PostgresDialectFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +public class PostgresDialectFactoryTest { + + @Test + public void testPostgresDialectCreate() { + PostgresDialectFactory postgresDialectFactory = new PostgresDialectFactory(); + JdbcDialect postgresLow = postgresDialectFactory.create("postgresLow"); + String[] fields = {"id", "name", "age"}; + String[] uniqueKeyField = {"id"}; + Optional upsertStatement = + postgresLow.getUpsertStatement("test", "test_a", fields, uniqueKeyField); + Assertions.assertFalse(upsertStatement.isPresent()); + } +} From 00e0ce9e221ec4abde133ebe92db345638e17702 Mon Sep 17 00:00:00 2001 From: XiaoJiang521 <131635688+XiaoJiang521@users.noreply.github.com> Date: Wed, 2 Aug 2023 19:00:49 +0800 Subject: [PATCH 40/43] [e2e] kafka e2e error (#5200) --- .../e2e/connector/kafka/DebeziumToKafkaIT.java | 11 +++++++++++ .../resources/kafkasource_debezium_cdc_to_pgsql.conf | 4 ++-- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/DebeziumToKafkaIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/DebeziumToKafkaIT.java index e76a4459963c..da1ee137192b 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/DebeziumToKafkaIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/java/org/apache/seatunnel/e2e/connector/kafka/DebeziumToKafkaIT.java @@ -314,6 +314,17 @@ public void testDebeziumFormatKafkaCdcToPgsql(TestContainer container) 108, "jacket", "water resistent black wind breaker", "0.1")) .collect(Collectors.toSet()); Assertions.assertIterableEquals(expected, actual); + + try (Connection connection = + DriverManager.getConnection( + POSTGRESQL_CONTAINER.getJdbcUrl(), + POSTGRESQL_CONTAINER.getUsername(), + POSTGRESQL_CONTAINER.getPassword())) { + try (Statement statement = connection.createStatement()) { + statement.execute("truncate table sink"); + LOG.info("testDebeziumFormatKafkaCdcToPgsql truncate table sink"); + } + } } public void initializeSourceTableData() throws Exception { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_cdc_to_pgsql.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_cdc_to_pgsql.conf index a0531b2345a0..2d56fb7879d6 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_cdc_to_pgsql.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-kafka-e2e/src/test/resources/kafkasource_debezium_cdc_to_pgsql.conf @@ -55,8 +55,8 @@ sink { user = test password = test generate_sink_sql = true - database = public - table = sink + database = test + table = public.sink primary_keys = ["id"] } } \ No newline at end of file From 7b6be590e118c3eb08d9cb1bb48cdb0a89f65483 Mon Sep 17 00:00:00 2001 From: Volodymyr Date: Thu, 3 Aug 2023 18:12:05 +0800 Subject: [PATCH 41/43] [Hotfix][Connector-V2][JindoOssFile] Fix plugin-mapping.properties (#5215) Co-authored-by: tyrantlucifer --- plugin-mapping.properties | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plugin-mapping.properties b/plugin-mapping.properties index a1c4e40fbb08..551da2c7cecf 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -47,8 +47,8 @@ seatunnel.source.LocalFile = connector-file-local seatunnel.sink.LocalFile = connector-file-local seatunnel.source.OssFile = connector-file-oss seatunnel.sink.OssFile = connector-file-oss -seatunnel.source.OssJindoFile = connector-file-oss-jindo -seatunnel.sink.OssJindoFile = connector-file-oss-jindo +seatunnel.source.OssJindoFile = connector-file-jindo-oss +seatunnel.sink.OssJindoFile = connector-file-jindo-oss seatunnel.source.CosFile = connector-file-cos seatunnel.sink.CosFile = connector-file-cos seatunnel.source.Pulsar = connector-pulsar From a6157cb38fc9be6140be349f4846bbede4da74c4 Mon Sep 17 00:00:00 2001 From: Jia Fan Date: Thu, 3 Aug 2023 18:18:54 +0800 Subject: [PATCH 42/43] [Improve][Zeta] Don't trigger handleSaveMode when restore (#5192) --- .../engine/client/job/JobExecutionEnvironment.java | 2 +- .../engine/core/parse/JobConfigParser.java | 12 +++++++++--- .../core/parse/MultipleTableJobConfigParser.java | 14 ++++++++++---- 3 files changed, 20 insertions(+), 8 deletions(-) diff --git a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobExecutionEnvironment.java b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobExecutionEnvironment.java index 9f28f6fdbb0c..bf3169e4c803 100644 --- a/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobExecutionEnvironment.java +++ b/seatunnel-engine/seatunnel-engine-client/src/main/java/org/apache/seatunnel/engine/client/job/JobExecutionEnvironment.java @@ -131,7 +131,7 @@ private Set searchPluginJars() { private MultipleTableJobConfigParser getJobConfigParser() { return new MultipleTableJobConfigParser( - jobFilePath, idGenerator, jobConfig, commonPluginJars); + jobFilePath, idGenerator, jobConfig, commonPluginJars, isStartWithSavePoint); } private LogicalDagGenerator getLogicalDagGenerator() { diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/JobConfigParser.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/JobConfigParser.java index d81de1702ed0..09bae74f5a24 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/JobConfigParser.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/JobConfigParser.java @@ -63,12 +63,16 @@ public class JobConfigParser { private static final ILogger LOGGER = Logger.getLogger(JobConfigParser.class); private IdGenerator idGenerator; - + private boolean isStartWithSavePoint; private List commonPluginJars; - public JobConfigParser(@NonNull IdGenerator idGenerator, @NonNull List commonPluginJars) { + public JobConfigParser( + @NonNull IdGenerator idGenerator, + @NonNull List commonPluginJars, + boolean isStartWithSavePoint) { this.idGenerator = idGenerator; this.commonPluginJars = commonPluginJars; + this.isStartWithSavePoint = isStartWithSavePoint; } public Tuple2 parseSource( @@ -190,7 +194,9 @@ public Tuple2 parseTransform( sink.prepare(config); sink.setJobContext(jobConfig.getJobContext()); sink.setTypeInfo(rowType); - handleSaveMode(sink); + if (!isStartWithSavePoint) { + handleSaveMode(sink); + } final String actionName = createSinkActionName(0, tuple.getLeft().getPluginName(), getTableName(config)); final SinkAction action = diff --git a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java index 09027a2a248f..86c0f3c94f59 100644 --- a/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java +++ b/seatunnel-engine/seatunnel-engine-core/src/main/java/org/apache/seatunnel/engine/core/parse/MultipleTableJobConfigParser.java @@ -107,23 +107,27 @@ public class MultipleTableJobConfigParser { private final ReadonlyConfig envOptions; private final JobConfigParser fallbackParser; + private final boolean isStartWithSavePoint; public MultipleTableJobConfigParser( String jobDefineFilePath, IdGenerator idGenerator, JobConfig jobConfig) { - this(jobDefineFilePath, idGenerator, jobConfig, Collections.emptyList()); + this(jobDefineFilePath, idGenerator, jobConfig, Collections.emptyList(), false); } public MultipleTableJobConfigParser( String jobDefineFilePath, IdGenerator idGenerator, JobConfig jobConfig, - List commonPluginJars) { + List commonPluginJars, + boolean isStartWithSavePoint) { this.idGenerator = idGenerator; this.jobConfig = jobConfig; this.commonPluginJars = commonPluginJars; + this.isStartWithSavePoint = isStartWithSavePoint; this.seaTunnelJobConfig = ConfigBuilder.of(Paths.get(jobDefineFilePath)); this.envOptions = ReadonlyConfig.fromConfig(seaTunnelJobConfig.getConfig("env")); - this.fallbackParser = new JobConfigParser(idGenerator, commonPluginJars); + this.fallbackParser = + new JobConfigParser(idGenerator, commonPluginJars, isStartWithSavePoint); } public ImmutablePair, Set> parse() { @@ -607,7 +611,9 @@ private static T findLast(LinkedHashMap map) { sink, factoryUrls, actionConfig); - handleSaveMode(sink); + if (!isStartWithSavePoint) { + handleSaveMode(sink); + } sinkAction.setParallelism(parallelism); return sinkAction; } From 573306bfb312823722dc695f779665e19dda866c Mon Sep 17 00:00:00 2001 From: Xiaojian Sun Date: Fri, 4 Aug 2023 11:17:18 +0800 Subject: [PATCH 43/43] move imap storage file dependency packages to submodules (#5218) --- pom.xml | 38 ------------------- .../src/main/assembly/assembly-bin-ci.xml | 1 + .../imap-storage-file/pom.xml | 19 +++++++--- tools/dependencies/known-dependencies.txt | 5 +-- 4 files changed, 16 insertions(+), 47 deletions(-) diff --git a/pom.xml b/pom.xml index 7dce624be315..3d619644952b 100644 --- a/pom.xml +++ b/pom.xml @@ -141,11 +141,6 @@ 4.2.0 true - - 3.0.0 - 2.4.7 - 3.1.4 - 4.1.60.Final
@@ -452,39 +447,6 @@ provided - - - org.apache.hadoop - hadoop-aliyun - ${hadoop-aliyun.version} - provided - - - net.minidev - json-smart - - - - - - net.minidev - json-smart - ${json-smart.version} - - - - org.apache.hadoop - hadoop-aws - ${hadoop-aws.version} - provided - - - - io.netty - netty-buffer - ${netty-buffer.version} - - diff --git a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml index 3a1b5a4177ed..de11af1e173c 100644 --- a/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml +++ b/seatunnel-dist/src/main/assembly/assembly-bin-ci.xml @@ -178,6 +178,7 @@ io.netty:netty-buffer:jar io.netty:netty-common:jar + ${artifact.file.name} /lib diff --git a/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/pom.xml b/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/pom.xml index c39ddda99c85..a742fe39a0b9 100644 --- a/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/pom.xml +++ b/seatunnel-engine/seatunnel-engine-storage/imap-storage-plugins/imap-storage-file/pom.xml @@ -30,6 +30,14 @@ imap-storage-file SeaTunnel : Engine : Storage : IMap Storage Plugins : File + + + 3.0.0 + 2.4.7 + 3.1.4 + 4.1.60.Final + + org.apache.seatunnel @@ -64,24 +72,25 @@ awaitility + org.apache.hadoop hadoop-aliyun - - - - net.minidev - json-smart + ${hadoop-aliyun.version} + provided org.apache.hadoop hadoop-aws + ${hadoop-aws.version} + provided io.netty netty-buffer + ${netty-buffer.version} provided diff --git a/tools/dependencies/known-dependencies.txt b/tools/dependencies/known-dependencies.txt index 70bbd1c0df53..11e5c57b8af9 100755 --- a/tools/dependencies/known-dependencies.txt +++ b/tools/dependencies/known-dependencies.txt @@ -34,7 +34,4 @@ j2objc-annotations-1.1.jar jsr305-1.3.9.jar jsr305-3.0.0.jar jsr305-3.0.2.jar -listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar -accessors-smart-2.4.7.jar -asm-9.1.jar -json-smart-2.4.7.jar +listenablefuture-9999.0-empty-to-avoid-conflict-with-guava.jar \ No newline at end of file