diff --git a/docs/en/connector-v2/sink/MongoDB.md b/docs/en/connector-v2/sink/MongoDB.md index eb02f5eacd3..b1530a3ac9c 100644 --- a/docs/en/connector-v2/sink/MongoDB.md +++ b/docs/en/connector-v2/sink/MongoDB.md @@ -1,47 +1,222 @@ # MongoDB -> MongoDB sink connector +> MongoDB Sink Connector -## Description +Support Those Engines +--------------------- -Write data to `MongoDB` +> Spark
+> Flink
+> SeaTunnel Zeta
-## Key features +Key Features +------------ - [ ] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [cdc](../../concept/connector-v2-features.md) + +Description +----------- + +The MongoDB Connector provides the ability to read and write data from and to MongoDB. +This document describes how to set up the MongoDB connector to run data writers against MongoDB. + +Supported DataSource Info +------------------------- + +In order to use the Mongodb 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-mongodb) | + +Data Type Mapping +----------------- + +The following table lists the field data type mapping from MongoDB BSON type to Seatunnel data type. + +| Seatunnel Data Type | MongoDB BSON Type | +|---------------------|-------------------| +| STRING | ObjectId | +| STRING | String | +| BOOLEAN | Boolean | +| BINARY | Binary | +| INTEGER | Int32 | +| TINYINT | Int32 | +| SMALLINT | Int32 | +| BIGINT | Int64 | +| DOUBLE | Double | +| FLOAT | Double | +| DECIMAL | Decimal128 | +| Date | Date | +| Timestamp | Timestamp[Date] | +| ROW | Object | +| ARRAY | Array | + +**Tips** + +> 1.When using SeaTunnel to write Date and Timestamp types to MongoDB, both will produce a Date data type in MongoDB, but the precision will be different. The data generated by the SeaTunnel Date type has second-level precision, while the data generated by the SeaTunnel Timestamp type has millisecond-level precision.
+> 2.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).
+ +Sink Options +------------ + +| Name | Type | Required | Default | Description | +|-----------------------|----------|----------|---------|----------------------------------------------------------------------------------------------------------------| +| uri | String | Yes | - | The MongoDB connection uri. | +| database | String | Yes | - | The name of MongoDB database to read or write. | +| collection | String | Yes | - | The name of MongoDB collection to read or write. | +| schema | String | Yes | - | MongoDB's BSON and seatunnel data structure mapping | +| buffer-flush.max-rows | String | No | 1000 | Specifies the maximum number of buffered rows per batch request. | +| buffer-flush.interval | String | No | 30000 | Specifies the retry time interval if writing records to database failed, the unit is seconds. | +| retry.max | String | No | 3 | Specifies the max retry times if writing records to database failed. | +| retry.interval | Duration | No | 1000 | Specifies the retry time interval if writing records to database failed, the unit is millisecond. | +| upsert-enable | Boolean | No | false | Whether to write documents via upsert mode. | +| upsert-key | List | No | - | The primary keys for upsert. Only valid in upsert mode. Keys are in `["id","name",...]` format for properties. | + +**Tips** + +> 1.The data flushing logic of the MongoDB Sink Connector is jointly controlled by three parameters: `buffer-flush.max-rows`, `buffer-flush.interval`, and `checkpoint.interval`. +> Data flushing will be triggered if any of these conditions are met.
+ +How to Create a MongoDB Data Synchronization Jobs +------------------------------------------------- + +The following example demonstrates how to create a data synchronization job that writes randomly generated data to a MongoDB database: -## Options +```bash +# Set the basic configuration of the task to be performed +env { + execution.parallelism = 1 + job.mode = "BATCH" + checkpoint.interval = 1000 +} + +source { + FakeSource { + row.num = 2 + bigint.min = 0 + bigint.max = 10000000 + split.num = 1 + split.read-interval = 300 + schema { + fields { + c_bigint = bigint + } + } + } +} + +sink { + MongoDB{ + uri = mongodb://user:password@127.0.0.1:27017 + database = "test" + collection = "test" + schema = { + fields { + _id = string + c_bigint = bigint + } + } + } +} +``` + +Parameter Interpretation +------------------------ + +**MongoDB Database Connection URI Examples** + +Unauthenticated single node connection: + +```bash +mongodb://127.0.0.0:27017/mydb +``` + +Replica set connection: + +```bash +mongodb://127.0.0.0:27017/mydb?replicaSet=xxx +``` + +Authenticated replica set connection: -| name | type | required | default value | -|----------------|--------|----------|---------------| -| uri | string | yes | - | -| database | string | yes | - | -| collection | string | yes | - | -| common-options | config | no | - | +```bash +mongodb://admin:password@127.0.0.0:27017/mydb?replicaSet=xxx&authSource=admin +``` + +Multi-node replica set connection: + +```bash +mongodb://127.0.0..1:27017,127.0.0..2:27017,127.0.0.3:27017/mydb?replicaSet=xxx +``` + +Sharded cluster connection: + +```bash +mongodb://127.0.0.0:27017/mydb +``` -### uri [string] +Multiple mongos connections: -uri to write to mongoDB +```bash +mongodb://192.168.0.1:27017,192.168.0.2:27017,192.168.0.3:27017/mydb +``` -### database [string] +Note: The username and password in the URI must be URL-encoded before being concatenated into the connection string. -database to write to mongoDB +**Buffer Flush** -### collection [string] +```bash +sink { + MongoDB { + uri = "mongodb://user:password@127.0.0.1:27017" + database = "test_db" + collection = "users" + buffer-flush.max-rows = 2000 + buffer-flush.interval = 1000 + schema = { + fields { + _id = string + id = bigint + status = string + } + } + } +} +``` + +**Why is Not Recommended to Use Transactions for Operation?** -collection to write to mongoDB +Although MongoDB has fully supported multi-document transactions since version 4.2, it doesn't mean that everyone should use them recklessly. +Transactions are equivalent to locks, node coordination, additional overhead, and performance impact. +Instead, the principle for using transactions should be: avoid using them if possible. +The necessity for using transactions can be greatly avoided by designing systems rationally. -### common options +**Idempotent Writes** -Sink plugin common parameters, please refer to [Sink Common Options](common-options.md) for details +By specifying a clear primary key and using the upsert method, exactly-once write semantics can be achieved. -## Example +If upsert-key is defined in the configuration, the MongoDB sink will use upsert semantics instead of regular INSERT statements. We combine the primary keys declared in upsert-key as the MongoDB reserved primary key and use upsert mode for writing to ensure idempotent writes. +In the event of a failure, Seatunnel jobs will recover from the last successful checkpoint and reprocess, which may result in duplicate message processing during recovery. It is highly recommended to use upsert mode, as it helps to avoid violating database primary key constraints and generating duplicate data if records need to be reprocessed. ```bash -mongodb { - uri = "mongodb://username:password@127.0.0.1:27017/mypost?retryWrites=true&writeConcern=majority" - database = "mydatabase" - collection = "mycollection" +sink { + MongoDB { + uri = "mongodb://user:password@127.0.0.1:27017" + database = "test_db" + collection = "users" + upsert-enable = true + upsert-key = ["name","status"] + schema = { + fields { + _id = string + name = string + status = string + } + } + } } ``` @@ -49,5 +224,9 @@ mongodb { ### 2.2.0-beta 2022-09-26 -- Add MongoDB Sink Connector +- Add MongoDB Source Connector + +### Next Version + +- [Feature]Refactor mongodb source connector([4620](https://github.com/apache/incubator-seatunnel/pull/4620)) diff --git a/docs/en/connector-v2/source/MongoDB.md b/docs/en/connector-v2/source/MongoDB.md index affd2335da5..aeca7b3ae86 100644 --- a/docs/en/connector-v2/source/MongoDB.md +++ b/docs/en/connector-v2/source/MongoDB.md @@ -1,85 +1,444 @@ # MongoDB -> MongoDB source connector +> MongoDB Source Connector -## Description +Support Those Engines +--------------------- -Read data from MongoDB. +> 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] [exactly-once](../../concept/connector-v2-features.md) - [x] [column projection](../../concept/connector-v2-features.md) -- [ ] [parallelism](../../concept/connector-v2-features.md) -- [ ] [support user-defined split](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) -## Options +Description +----------- -| name | type | required | default value | -|----------------|--------|----------|---------------| -| uri | string | yes | - | -| database | string | yes | - | -| collection | string | yes | - | -| matchQuery | string | no | - | -| schema | object | yes | - | -| common-options | config | no | - | +The MongoDB Connector provides the ability to read and write data from and to MongoDB. +This document describes how to set up the MongoDB connector to run data reads against MongoDB. -### uri [string] +Supported DataSource Info +------------------------- -MongoDB uri +In order to use the Mongodb connector, the following dependencies are required. +They can be downloaded via install-plugin.sh or from the Maven central repository. -### database [string] +| Datasource | Supported Versions | Dependency | +|------------|--------------------|---------------------------------------------------------------------------------------------------------------| +| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-mongodb) | -MongoDB database +Data Type Mapping +----------------- -### collection [string] +The following table lists the field data type mapping from MongoDB BSON type to Seatunnel data type. -MongoDB collection +| 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 | -### matchQuery [string] +For specific types in MongoDB, we use Extended JSON format to map them to Seatunnel STRING type. -MatchQuery is a JSON string that specifies the selection criteria using query operators for the documents to be returned from the collection. +| 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"}}}} | -### schema [object] +**Tips** -#### fields [Config] +> 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).
-Because `MongoDB` does not have the concept of `schema`, when engine reads `MongoDB` , it will sample `MongoDB` data and infer the `schema` . In fact, this process will be slow and may be inaccurate. This parameter can be manually specified. Avoid these problems. +Source Options +-------------- -such as: +| Name | Type | Required | Default | Description | +|----------------------|---------|----------|------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| uri | String | Yes | - | The MongoDB connection uri. | +| database | String | Yes | - | The name of MongoDB database to read or write. | +| collection | String | Yes | - | The name of MongoDB collection to read or write. | +| schema | String | Yes | - | MongoDB's BSON and seatunnel data structure mapping | +| match.query | String | No | - | In MongoDB, filters are used to filter documents for query operations. | +| match.projection | String | No | - | In MongoDB, Projection is used to control the fields contained in the query results | +| partition.split-key | String | No | _id | The key of Mongodb fragmentation. | +| partition.split-size | Long | No | 64 * 1024 * 1024 | The size of Mongodb fragment. | +| cursor.no-timeout | Boolean | No | true | MongoDB server normally times out idle cursors after an inactivity period (10 minutes) to prevent excess memory use. Set this option to true to prevent that. However, if the application takes longer than 30 minutes to process the current batch of documents, the session is marked as expired and closed. | +| fetch.size | Int | No | 2048 | Set the number of documents obtained from the server for each batch. Setting the appropriate batch size can improve query performance and avoid the memory pressure caused by obtaining a large amount of data at one time. | +| max.time-min | Long | No | 600 | This parameter is a MongoDB query option that limits the maximum execution time for query operations. The value of maxTimeMin is in Minute. If the execution time of the query exceeds the specified time limit, MongoDB will terminate the operation and return an error. | +| flat.sync-string | Boolean | No | true | By utilizing flatSyncString, only one field attribute value can be set, and the field type must be a String. This operation will perform a string mapping on a single MongoDB data entry. | +How to Create a MongoDB Data Synchronization Jobs +------------------------------------------------- + +The following example demonstrates how to create a data synchronization job that reads data from MongoDB 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" +} + +# Create a source to connect to Mongodb +source { + MongoDB { + uri = "mongodb://user:password@127.0.0.1:27017" + database = "test_db" + collection = "source_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + 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_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(38, 18)" + c_timestamp = timestamp + } + } + } + } +} + +# Console printing of the read Mongodb data +sink { + Console { + parallelism = 1 + } +} +``` + +Parameter Interpretation +------------------------ + +**MongoDB Database Connection URI Examples** + +Unauthenticated single node connection: + +```bash +mongodb://192.168.0.100:27017/mydb +``` + +Replica set connection: + +```bash +mongodb://192.168.0.100:27017/mydb?replicaSet=xxx +``` + +Authenticated replica set connection: + +```bash +mongodb://admin:password@192.168.0.100:27017/mydb?replicaSet=xxx&authSource=admin +``` + +Multi-node replica set connection: + +```bash +mongodb://192.168.0.1:27017,192.168.0.2:27017,192.168.0.3:27017/mydb?replicaSet=xxx +``` + +Sharded cluster connection: + +```bash +mongodb://192.168.0.100:27017/mydb +``` + +Multiple mongos connections: + +```bash +mongodb://192.168.0.1:27017,192.168.0.2:27017,192.168.0.3:27017/mydb +``` + +Note: The username and password in the URI must be URL-encoded before being concatenated into the connection string. + +**MatchQuery Scan** + +In data synchronization scenarios, the matchQuery approach needs to be used early to reduce the number of documents that need to be processed by subsequent operators, thus improving performance. +Here is a simple example of a seatunnel using `match.query` + +```bash +source { + MongoDB { + uri = "mongodb://user:password@127.0.0.1:27017" + database = "test_db" + collection = "orders" + match.query = "{status: \"A\"}" + schema = { + fields { + id = bigint + status = string + } + } + } +} +``` + +The following are examples of MatchQuery query statements of various data types: + +```bash +# Query Boolean type +"{c_boolean:true}" +# Query string type +"{c_string:\"OCzCj\"}" +# Query the integer +"{c_int:2}" +# Type of query time +"{c_date:ISODate(\"2023-06-26T16:00:00.000Z\")}" +# Query floating point type +{c_double:{$gte:1.71763202185342e+308}} ``` -schema { - fields { - id = int - key_aa = string - key_bb = string + +Please refer to how to write the syntax of `match.query`:https://www.mongodb.com/docs/manual/tutorial/query-documents + +**Projection Scan** + +In MongoDB, Projection is used to control which fields are included in the query results. This can be accomplished by specifying which fields need to be returned and which fields do not. +In the find() method, a projection object can be passed as a second argument. The key of the projection object indicates the fields to include or exclude, and a value of 1 indicates inclusion and 0 indicates exclusion. +Here is a simple example, assuming we have a collection named users: + +```bash +# Returns only the name and email fields +db.users.find({}, { name: 1, email: 0 }); +``` + +In data synchronization scenarios, projection needs to be used early to reduce the number of documents that need to be processed by subsequent operators, thus improving performance. +Here is a simple example of a seatunnel using projection: + +```bash +source { + MongoDB { + uri = "mongodb://user:password@127.0.0.1:27017" + database = "test_db" + collection = "users" + match.projection = "{ name: 1, email: 0 }" + schema = { + fields { + name = string + } + } } } + ``` -### common options +**Partitioned Scan** + +To speed up reading data in parallel source task instances, seatunnel provides a partitioned scan feature for MongoDB collections. The following partitioning strategies are provided. +Users can control data sharding by setting the partition.split-key for sharding keys and partition.split-size for sharding size. + +```bash +source { + MongoDB { + uri = "mongodb://user:password@127.0.0.1:27017" + database = "test_db" + collection = "users" + partition.split-key = "id" + partition.split-size = 1024 + schema = { + fields { + id = bigint + status = string + } + } + } +} -Source Plugin common parameters, refer to [Source Plugin](common-options.md) for details +``` + +**Flat Sync String** -## Example +By utilizing `flat.sync-string`, only one field attribute value can be set, and the field type must be a String. +This operation will perform a string mapping on a single MongoDB data entry. ```bash -mongodb { - uri = "mongodb://username:password@127.0.0.1:27017/mypost?retryWrites=true&writeConcern=majority" - database = "mydatabase" - collection = "mycollection" - matchQuery = "{"id":3}" - schema { +source { + MongoDB { + uri = "mongodb://user:password@127.0.0.1:27017" + database = "test_db" + collection = "users" + flat.sync-string = true + schema = { fields { - id = int - key_aa = string - key_bb = string + data = string } } - result_table_name = "mongodb_result_table" + } +} +``` + +Use the data samples synchronized with modified parameters, such as the following: + +```json +{ + "_id":{ + "$oid":"643d41f5fdc6a52e90e59cbf" + }, + "c_map":{ + "OQBqH":"jllt", + "rkvlO":"pbfdf", + "pCMEX":"hczrdtve", + "DAgdj":"t", + "dsJag":"voo" + }, + "c_array":[ + { + "$numberInt":"-865590937" + }, + { + "$numberInt":"833905600" + }, + { + "$numberInt":"-1104586446" + }, + { + "$numberInt":"2076336780" + }, + { + "$numberInt":"-1028688944" + } + ], + "c_string":"bddkzxr", + "c_boolean":false, + "c_tinyint":{ + "$numberInt":"39" + }, + "c_smallint":{ + "$numberInt":"23672" + }, + "c_int":{ + "$numberInt":"-495763561" + }, + "c_bigint":{ + "$numberLong":"3768307617923954543" + }, + "c_float":{ + "$numberDouble":"5.284220288280258E37" + }, + "c_double":{ + "$numberDouble":"1.1706091642478246E308" + }, + "c_bytes":{ + "$binary":{ + "base64":"ZWJ4", + "subType":"00" + } + }, + "c_date":{ + "$date":{ + "$numberLong":"1686614400000" + } + }, + "c_decimal":{ + "$numberDecimal":"683265300" + }, + "c_timestamp":{ + "$date":{ + "$numberLong":"1684283772000" + } + }, + "c_row":{ + "c_map":{ + "OQBqH":"cbrzhsktmm", + "rkvlO":"qtaov", + "pCMEX":"tuq", + "DAgdj":"jzop", + "dsJag":"vwqyxtt" + }, + "c_array":[ + { + "$numberInt":"1733526799" + }, + { + "$numberInt":"-971483501" + }, + { + "$numberInt":"-1716160960" + }, + { + "$numberInt":"-919976360" + }, + { + "$numberInt":"727499700" + } + ], + "c_string":"oboislr", + "c_boolean":true, + "c_tinyint":{ + "$numberInt":"-66" + }, + "c_smallint":{ + "$numberInt":"1308" + }, + "c_int":{ + "$numberInt":"-1573886733" + }, + "c_bigint":{ + "$numberLong":"4877994302999518682" + }, + "c_float":{ + "$numberDouble":"1.5353209063652051E38" + }, + "c_double":{ + "$numberDouble":"1.1952441956458565E308" + }, + "c_bytes":{ + "$binary":{ + "base64":"cWx5Ymp0Yw==", + "subType":"00" + } + }, + "c_date":{ + "$date":{ + "$numberLong":"1686614400000" + } + }, + "c_decimal":{ + "$numberDecimal":"656406177" + }, + "c_timestamp":{ + "$date":{ + "$numberLong":"1684283772000" + } + } + }, + "id":{ + "$numberInt":"2" + } } ``` @@ -91,5 +450,5 @@ mongodb { ### Next Version -- common-options is not a required option +- [Feature]Refactor mongodb source connector([4620](https://github.com/apache/incubator-seatunnel/pull/4620)) diff --git a/seatunnel-connectors-v2/connector-mongodb/pom.xml b/seatunnel-connectors-v2/connector-mongodb/pom.xml index 31169000e03..dd05b1d7478 100644 --- a/seatunnel-connectors-v2/connector-mongodb/pom.xml +++ b/seatunnel-connectors-v2/connector-mongodb/pom.xml @@ -30,7 +30,7 @@ SeaTunnel : Connectors V2 : Mongodb - 3.12.11 + 4.7.1 @@ -48,8 +48,20 @@ org.mongodb - mongodb-driver - ${mongodb.version} + mongodb-driver-sync + ${mongo.driver.version} + + + + org.mongodb + mongodb-driver-core + ${mongo.driver.version} + + + + org.mongodb + bson + ${mongo.driver.version} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/config/MongodbConfig.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/config/MongodbConfig.java index 53b98d18385..0efe88ad0f5 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/config/MongodbConfig.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/config/MongodbConfig.java @@ -17,42 +17,135 @@ package org.apache.seatunnel.connectors.seatunnel.mongodb.config; -import org.apache.seatunnel.shade.com.typesafe.config.Config; - -import lombok.Builder; -import lombok.Getter; - -import java.io.Serializable; - -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.COLLECTION; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.DATABASE; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.MATCHQUERY; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.URI; - -/** The config of mongodb */ -@Builder -@Getter -public class MongodbConfig implements Serializable { - - @Builder.Default private String uri = URI.defaultValue(); - @Builder.Default private String database = DATABASE.defaultValue(); - @Builder.Default private String collection = COLLECTION.defaultValue(); - @Builder.Default private String matchQuery = MATCHQUERY.defaultValue(); - - public static MongodbConfig buildWithConfig(Config config) { - MongodbConfigBuilder builder = MongodbConfig.builder(); - if (config.hasPath(URI.key())) { - builder.uri(config.getString(URI.key())); - } - if (config.hasPath(DATABASE.key())) { - builder.database(config.getString(DATABASE.key())); - } - if (config.hasPath(COLLECTION.key())) { - builder.collection(config.getString(COLLECTION.key())); - } - if (config.hasPath(MATCHQUERY.key())) { - builder.matchQuery(config.getString(MATCHQUERY.key())); - } - return builder.build(); - } +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +import org.bson.json.JsonMode; +import org.bson.json.JsonWriterSettings; + +import java.util.List; + +public class MongodbConfig { + + public static final String CONNECTOR_IDENTITY = "MongoDB"; + + public static final String ENCODE_VALUE_FIELD = "_value"; + + public static final JsonWriterSettings DEFAULT_JSON_WRITER_SETTINGS = + JsonWriterSettings.builder().outputMode(JsonMode.EXTENDED).build(); + + public static final Option URI = + Options.key("uri") + .stringType() + .noDefaultValue() + .withDescription("The MongoDB connection uri."); + + public static final Option DATABASE = + Options.key("database") + .stringType() + .noDefaultValue() + .withDescription("The name of MongoDB database to read or write."); + + public static final Option COLLECTION = + Options.key("collection") + .stringType() + .noDefaultValue() + .withDescription("The name of MongoDB collection to read or write."); + + public static final Option MATCH_QUERY = + Options.key("match.query") + .stringType() + .noDefaultValue() + .withDescription("Mongodb's query syntax."); + + public static final Option PROJECTION = + Options.key("match.projection") + .stringType() + .noDefaultValue() + .withDescription("Fields projection by Mongodb."); + + public static final Option SPLIT_KEY = + Options.key("partition.split-key") + .stringType() + .defaultValue("_id") + .withDescription("The key of Mongodb fragmentation."); + + public static final Option SPLIT_SIZE = + Options.key("partition.split-size") + .longType() + .defaultValue(64 * 1024 * 1024L) + .withDescription("The size of Mongodb fragment."); + + public static final Option FETCH_SIZE = + Options.key("fetch.size") + .intType() + .defaultValue(2048) + .withDescription( + "Set the number of documents obtained from the server for each batch. Setting the appropriate batch size can improve query performance and avoid the memory pressure caused by obtaining a large amount of data at one time."); + + public static final Option CURSOR_NO_TIMEOUT = + Options.key("cursor.no-timeout") + .booleanType() + .defaultValue(true) + .withDescription( + "MongoDB server normally times out idle cursors after an inactivity period (10 minutes) to prevent excess memory use. Set this option to true to prevent that. However, if the application takes longer than 30 minutes to process the current batch of documents, the session is marked as expired and closed."); + + public static final Option MAX_TIME_MIN = + Options.key("max.time-min") + .longType() + .defaultValue(600L) + .withDescription( + "This parameter is a MongoDB query option that limits the maximum execution time for query operations. The value of maxTimeMS is in milliseconds. If the execution time of the query exceeds the specified time limit, MongoDB will terminate the operation and return an error."); + + public static final Option FLAT_SYNC_STRING = + Options.key("flat.sync-string") + .booleanType() + .defaultValue(false) + .withDescription( + "By utilizing flatSyncString, only one field attribute value can be set, and the field type must be a String. This operation will perform a string mapping on a single MongoDB data entry."); + + // -------------------------------------------------------------------- + // The following are the sink parameters. + // -------------------------------------------------------------------- + + public static final Option BUFFER_FLUSH_MAX_ROWS = + Options.key("buffer-flush.max-rows") + .intType() + .defaultValue(1000) + .withDescription( + "Specifies the maximum number of buffered rows per batch request."); + + public static final Option BUFFER_FLUSH_INTERVAL = + Options.key("buffer-flush.interval") + .longType() + .defaultValue(30_000L) + .withDescription( + "Specifies the retry time interval if writing records to database failed."); + + public static final Option RETRY_MAX = + Options.key("retry.max") + .intType() + .defaultValue(3) + .withDescription( + "Specifies the max retry times if writing records to database failed."); + + public static final Option RETRY_INTERVAL = + Options.key("retry.interval") + .longType() + .defaultValue(1000L) + .withDescription( + "Specifies the retry time interval if writing records to database failed."); + + public static final Option UPSERT_ENABLE = + Options.key("upsert-enable") + .booleanType() + .defaultValue(false) + .withDescription("Whether to write documents via upsert mode."); + + public static final Option> UPSERT_KEY = + Options.key("upsert-key") + .listType() + .noDefaultValue() + .withDescription( + "The primary keys for upsert. Only valid in upsert mode. Keys are in csv format for properties."); } diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/config/MongodbOption.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/config/MongodbOption.java deleted file mode 100644 index 1675a5d9648..00000000000 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/config/MongodbOption.java +++ /dev/null @@ -1,51 +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.mongodb.config; - -import org.apache.seatunnel.api.configuration.Option; -import org.apache.seatunnel.api.configuration.Options; - -public class MongodbOption { - public static final Option URI = - Options.key("uri").stringType().noDefaultValue().withDescription("MongoDB uri"); - - public static final Option DATABASE = - Options.key("database") - .stringType() - .noDefaultValue() - .withDescription("MongoDB database name"); - - public static final Option COLLECTION = - Options.key("collection") - .stringType() - .noDefaultValue() - .withDescription("MongoDB collection"); - - public static final Option MATCHQUERY = - Options.key("matchQuery") - .stringType() - .noDefaultValue() - .withDescription( - "MatchQuery is a JSON string that specifies the selection criteria using query operators for the documents to be returned from the collection.\n"); - - // Don't use now - public static final String FORMAT = "format"; - - // Don't use now - public static final String DEFAULT_FORMAT = "json"; -} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/DataTypeValidator.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/DataTypeValidator.java deleted file mode 100644 index 081469e1be0..00000000000 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/DataTypeValidator.java +++ /dev/null @@ -1,53 +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.mongodb.data; - -import org.apache.seatunnel.api.table.type.MapType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -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.mongodb.exception.MongodbConnectorException; - -public class DataTypeValidator { - - public static void validateDataType(SeaTunnelDataType dataType) - throws IllegalArgumentException { - switch (dataType.getSqlType()) { - case TIME: - throw new MongodbConnectorException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, - "Unsupported data type: " + dataType); - case MAP: - MapType mapType = (MapType) dataType; - if (!SqlType.STRING.equals(mapType.getKeyType().getSqlType())) { - throw new MongodbConnectorException( - CommonErrorCode.UNSUPPORTED_DATA_TYPE, - "Unsupported map key type: " + mapType.getKeyType()); - } - break; - case ROW: - SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; - for (int i = 0; i < rowType.getTotalFields(); i++) { - validateDataType(rowType.getFieldType(i)); - } - break; - default: - } - } -} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/DefaultDeserializer.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/DefaultDeserializer.java deleted file mode 100644 index 81f085aca65..00000000000 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/DefaultDeserializer.java +++ /dev/null @@ -1,56 +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.mongodb.data; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; - -import org.bson.BsonWriter; -import org.bson.Document; -import org.bson.codecs.DocumentCodec; -import org.bson.codecs.Encoder; -import org.bson.codecs.EncoderContext; - -import lombok.NonNull; - -public class DefaultDeserializer implements Deserializer { - - private final SeaTunnelRowType rowType; - private final Encoder encoder; - - public DefaultDeserializer(@NonNull SeaTunnelRowType rowType) { - DataTypeValidator.validateDataType(rowType); - this.rowType = rowType; - this.encoder = new DocumentCodec(); - } - - @Override - public SeaTunnelRow deserialize(Document document) { - return convert(document); - } - - private SeaTunnelRow convert(Document document) { - SeaTunnelRow row = new SeaTunnelRow(rowType.getTotalFields()); - - BsonWriter writer = new SeaTunnelRowBsonWriter(rowType, row); - encoder.encode(writer, document, EncoderContext.builder().build()); - writer.flush(); - - return row; - } -} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/DefaultSerializer.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/DefaultSerializer.java deleted file mode 100644 index 0581dab8d80..00000000000 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/DefaultSerializer.java +++ /dev/null @@ -1,124 +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.mongodb.data; - -import org.apache.seatunnel.api.table.type.ArrayType; -import org.apache.seatunnel.api.table.type.MapType; -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.bson.BsonTimestamp; -import org.bson.Document; -import org.bson.types.Binary; -import org.bson.types.Decimal128; - -import lombok.NonNull; - -import java.math.BigDecimal; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.ArrayList; -import java.util.Date; -import java.util.List; -import java.util.Map; - -public class DefaultSerializer implements Serializer { - - private final SeaTunnelRowType rowType; - - public DefaultSerializer(@NonNull SeaTunnelRowType rowType) { - DataTypeValidator.validateDataType(rowType); - this.rowType = rowType; - } - - @Override - public Document serialize(@NonNull SeaTunnelRow row) { - return convert(rowType, row); - } - - private static Document convert(SeaTunnelRowType rowType, SeaTunnelRow row) { - Document document = new Document(); - for (int i = 0; i < rowType.getTotalFields(); i++) { - String fieldName = rowType.getFieldName(i); - SeaTunnelDataType fieldType = rowType.getFieldType(i); - Object fieldValue = row.getField(i); - document.append(fieldName, convert(fieldType, fieldValue)); - } - return document; - } - - private static Object convert(SeaTunnelDataType fieldType, Object fieldValue) { - if (fieldValue == null) { - return null; - } - switch (fieldType.getSqlType()) { - case TINYINT: - case SMALLINT: - Number number = (Number) fieldValue; - return number.intValue(); - case FLOAT: - Float floatValue = (Float) fieldValue; - return Double.parseDouble(String.valueOf(floatValue)); - case DECIMAL: - BigDecimal bigDecimal = (BigDecimal) fieldValue; - return new Decimal128(bigDecimal); - case DATE: - LocalDate localDate = (LocalDate) fieldValue; - return Date.from(localDate.atStartOfDay(ZoneOffset.UTC).toInstant()); - case TIMESTAMP: - LocalDateTime localDateTime = (LocalDateTime) fieldValue; - return new BsonTimestamp(localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli()); - case BYTES: - byte[] bytes = (byte[]) fieldValue; - return new Binary(bytes); - case ARRAY: - ArrayType arrayType = (ArrayType) fieldType; - Object[] array = (Object[]) fieldValue; - List listValues = new ArrayList(); - for (Object item : array) { - listValues.add(convert(arrayType.getElementType(), item)); - } - return listValues; - case MAP: - MapType mapType = (MapType) fieldType; - Map map = (Map) fieldValue; - Document mapDocument = new Document(); - for (Map.Entry entry : map.entrySet()) { - String mapKeyName = entry.getKey(); - mapDocument.append( - mapKeyName, convert(mapType.getValueType(), entry.getValue())); - } - return mapDocument; - case ROW: - SeaTunnelRowType rowType = (SeaTunnelRowType) fieldType; - SeaTunnelRow row = (SeaTunnelRow) fieldValue; - Document rowDocument = new Document(); - for (int i = 0; i < rowType.getTotalFields(); i++) { - String rowFieldName = rowType.getFieldName(i); - SeaTunnelDataType rowFieldType = rowType.getFieldType(i); - Object rowValue = row.getField(i); - rowDocument.append(rowFieldName, convert(rowFieldType, rowValue)); - } - return rowDocument; - default: - return fieldValue; - } - } -} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/SeaTunnelRowBsonWriter.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/SeaTunnelRowBsonWriter.java deleted file mode 100644 index 3f7045b447d..00000000000 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/SeaTunnelRowBsonWriter.java +++ /dev/null @@ -1,332 +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.mongodb.data; - -import org.apache.seatunnel.api.table.type.ArrayType; -import org.apache.seatunnel.api.table.type.MapType; -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.connectors.seatunnel.mongodb.exception.MongodbConnectorException; - -import org.bson.AbstractBsonWriter; -import org.bson.BsonBinary; -import org.bson.BsonContextType; -import org.bson.BsonDbPointer; -import org.bson.BsonRegularExpression; -import org.bson.BsonTimestamp; -import org.bson.BsonWriterSettings; -import org.bson.types.Decimal128; -import org.bson.types.ObjectId; - -import lombok.Getter; -import lombok.SneakyThrows; - -import java.lang.reflect.Array; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.ArrayList; -import java.util.Date; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class SeaTunnelRowBsonWriter extends AbstractBsonWriter { - - private final SeaTunnelRowType rowType; - private final SeaTunnelRow row; - - public SeaTunnelRowBsonWriter(SeaTunnelRowType rowType, SeaTunnelRow row) { - super(new BsonWriterSettings()); - this.rowType = rowType; - this.row = row; - setContext(new SeaTunnelContext(BsonContextType.TOP_LEVEL)); - } - - @Override - protected SeaTunnelContext getContext() { - return (SeaTunnelContext) super.getContext(); - } - - @Override - protected void doWriteStartDocument() { - BsonContextType contextType = - (getState() == State.SCOPE_DOCUMENT) - ? BsonContextType.SCOPE_DOCUMENT - : BsonContextType.DOCUMENT; - if (BsonContextType.TOP_LEVEL == getContext().getContextType()) { - setContext(new SeaTunnelContext(getContext(), contextType, rowType, row)); - } else { - SeaTunnelDataType dataType = getSeaTunnelDataType(); - Object dataContainer = createDataContainer(dataType); - - setContext(new SeaTunnelContext(getContext(), contextType, dataType, dataContainer)); - } - } - - @Override - protected void doWriteEndDocument() { - if (getContext().getContextType() == BsonContextType.SCOPE_DOCUMENT) { - setContext(getContext().getParentContext()); - writeEndDocument(); - } else { - SeaTunnelContext childContext = getContext(); - setContext(childContext.getParentContext()); - - SeaTunnelContext currentContext = getContext(); - if (BsonContextType.TOP_LEVEL != currentContext.getContextType()) { - currentContext.write(getName(), childContext.getDataContainer()); - } - } - } - - @Override - protected void doWriteStartArray() { - SeaTunnelRowType parentDataType = (SeaTunnelRowType) getContext().getDataType(); - int index = parentDataType.indexOf(getName()); - ArrayType dataType = (ArrayType) parentDataType.getFieldType(index); - List dataContainer = (List) createDataContainer(dataType); - - setContext( - new SeaTunnelContext(getContext(), BsonContextType.ARRAY, dataType, dataContainer)); - } - - @SneakyThrows - @Override - protected void doWriteEndArray() { - SeaTunnelContext childContext = getContext(); - ArrayType dataType = (ArrayType) childContext.getDataType(); - setContext(childContext.getParentContext()); - - List dataContainer = (List) childContext.getDataContainer(); - Object[] arrayType = - (Object[]) Array.newInstance(dataType.getElementType().getTypeClass(), 0); - getContext().write(getName(), dataContainer.toArray(arrayType)); - } - - @Override - protected void doWriteObjectId(ObjectId value) { - getContext().write(getName(), value.toString()); - } - - @Override - protected void doWriteNull() { - getContext().write(getName(), null); - } - - @Override - protected void doWriteUndefined() { - getContext().write(getName(), null); - } - - @Override - protected void doWriteBinaryData(BsonBinary value) { - getContext().write(getName(), value.getData()); - } - - @Override - protected void doWriteBoolean(boolean value) { - getContext().write(getName(), value); - } - - @Override - protected void doWriteDouble(double value) { - SeaTunnelDataType fieldType = getSeaTunnelDataType(); - switch (fieldType.getSqlType()) { - case FLOAT: - getContext().write(getName(), Double.valueOf(value).floatValue()); - break; - case DOUBLE: - default: - getContext().write(getName(), value); - } - } - - @Override - protected void doWriteInt32(int value) { - SeaTunnelDataType fieldType = getSeaTunnelDataType(); - switch (fieldType.getSqlType()) { - case TINYINT: - getContext().write(getName(), Integer.valueOf(value).byteValue()); - break; - case SMALLINT: - getContext().write(getName(), Integer.valueOf(value).shortValue()); - break; - case INT: - default: - getContext().write(getName(), value); - } - } - - @Override - protected void doWriteInt64(long value) { - getContext().write(getName(), value); - } - - @Override - protected void doWriteDecimal128(Decimal128 value) { - getContext().write(getName(), value.bigDecimalValue()); - } - - @Override - protected void doWriteJavaScript(String value) { - getContext().write(getName(), value); - } - - @Override - protected void doWriteString(String value) { - getContext().write(getName(), value); - } - - @Override - protected void doWriteSymbol(String value) { - getContext().write(getName(), value); - } - - @Override - protected void doWriteDateTime(long value) { - SeaTunnelDataType fieldType = getSeaTunnelDataType(); - switch (fieldType.getSqlType()) { - case DATE: - LocalDate localDate = - new Date(value).toInstant().atZone(ZoneOffset.UTC).toLocalDate(); - getContext().write(getName(), localDate); - break; - case TIMESTAMP: - default: - LocalDateTime localDateTime = - new Date(value).toInstant().atZone(ZoneOffset.UTC).toLocalDateTime(); - getContext().write(getName(), localDateTime); - } - } - - @Override - protected void doWriteTimestamp(BsonTimestamp value) { - LocalDateTime localDateTime = - new Date(value.getValue()).toInstant().atZone(ZoneOffset.UTC).toLocalDateTime(); - getContext().write(getName(), localDateTime); - } - - @Override - protected void doWriteJavaScriptWithScope(String value) { - throw new MongodbConnectorException( - CommonErrorCode.UNSUPPORTED_OPERATION, "Unsupported JavaScriptWithScope"); - } - - @Override - protected void doWriteMaxKey() { - throw new MongodbConnectorException( - CommonErrorCode.UNSUPPORTED_OPERATION, "Unsupported MaxKey"); - } - - @Override - protected void doWriteMinKey() { - throw new MongodbConnectorException( - CommonErrorCode.UNSUPPORTED_OPERATION, "Unsupported MinKey"); - } - - @Override - protected void doWriteRegularExpression(BsonRegularExpression value) { - throw new MongodbConnectorException( - CommonErrorCode.UNSUPPORTED_OPERATION, "Unsupported BsonRegularExpression"); - } - - @Override - protected void doWriteDBPointer(BsonDbPointer value) { - throw new MongodbConnectorException( - CommonErrorCode.UNSUPPORTED_OPERATION, "Unsupported BsonDbPointer"); - } - - @Override - public void flush() {} - - private SeaTunnelDataType getSeaTunnelDataType() { - SeaTunnelDataType dataType = getContext().getDataType(); - switch (dataType.getSqlType()) { - case ARRAY: - ArrayType arrayType = (ArrayType) dataType; - return arrayType.getElementType(); - case MAP: - MapType mapType = (MapType) dataType; - return mapType.getValueType(); - case ROW: - default: - SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; - int index = rowType.indexOf(getName()); - return rowType.getFieldType(index); - } - } - - private static Object createDataContainer(SeaTunnelDataType dataType) { - switch (dataType.getSqlType()) { - case ARRAY: - return new ArrayList<>(); - case MAP: - return new HashMap<>(); - case ROW: - default: - SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; - return new SeaTunnelRow(rowType.getTotalFields()); - } - } - - @Getter - private class SeaTunnelContext extends Context { - private final SeaTunnelDataType dataType; - private final Object dataContainer; - - public SeaTunnelContext(final BsonContextType contextType) { - this(null, contextType, null, null); - } - - public SeaTunnelContext( - final SeaTunnelContext parentContext, - final BsonContextType contextType, - final SeaTunnelDataType dataType, - final Object dataContainer) { - super(parentContext, contextType); - this.dataType = dataType; - this.dataContainer = dataContainer; - } - - @Override - public SeaTunnelContext getParentContext() { - return (SeaTunnelContext) super.getParentContext(); - } - - public void write(String fieldName, Object value) { - switch (dataType.getSqlType()) { - case ARRAY: - ((List) dataContainer).add(value); - break; - case MAP: - ((Map) dataContainer).put(fieldName, value); - break; - case ROW: - default: - SeaTunnelRowType rowType = (SeaTunnelRowType) dataType; - SeaTunnelRow row = (SeaTunnelRow) dataContainer; - int index = rowType.indexOf(fieldName); - row.setField(index, value); - break; - } - } - } -} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/exception/MongodbConnectorException.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/exception/MongodbConnectorException.java index 65b0fd0d4c8..fd36ef48066 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/exception/MongodbConnectorException.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/exception/MongodbConnectorException.java @@ -30,8 +30,4 @@ public MongodbConnectorException( SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage, Throwable cause) { super(seaTunnelErrorCode, errorMessage, cause); } - - public MongodbConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, Throwable cause) { - super(seaTunnelErrorCode, cause); - } } diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/internal/MongodbClientProvider.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/internal/MongodbClientProvider.java new file mode 100644 index 00000000000..ee2cc91be3c --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/internal/MongodbClientProvider.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.mongodb.internal; + +import org.bson.BsonDocument; + +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; + +import java.io.Serializable; + +/** Provided for initiate and recreate {@link MongoClient}. */ +public interface MongodbClientProvider extends Serializable { + + /** + * Create one or get the current {@link MongoClient}. + * + * @return Current {@link MongoClient}. + */ + MongoClient getClient(); + + /** + * Get the default database. + * + * @return Current {@link MongoDatabase}. + */ + MongoDatabase getDefaultDatabase(); + + /** + * Get the default collection. + * + * @return Current {@link MongoCollection}. + */ + MongoCollection getDefaultCollection(); + + /** Close the underlying MongoDB connection. */ + void close(); +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/internal/MongodbCollectionProvider.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/internal/MongodbCollectionProvider.java new file mode 100644 index 00000000000..5ebdab91c70 --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/internal/MongodbCollectionProvider.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.mongodb.internal; + +import com.google.common.base.Preconditions; + +/** A builder class for creating {@link MongodbClientProvider}. */ +public class MongodbCollectionProvider { + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + + private String connectionString; + + private String database; + + private String collection; + + public Builder connectionString(String connectionString) { + this.connectionString = connectionString; + return this; + } + + public Builder database(String database) { + this.database = database; + return this; + } + + public Builder collection(String collection) { + this.collection = collection; + return this; + } + + public MongodbClientProvider build() { + Preconditions.checkNotNull(connectionString, "Connection string must not be null"); + Preconditions.checkNotNull(database, "Database must not be null"); + Preconditions.checkNotNull(collection, "Collection must not be null"); + return new MongodbSingleCollectionProvider(connectionString, database, collection); + } + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/internal/MongodbSingleCollectionProvider.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/internal/MongodbSingleCollectionProvider.java new file mode 100644 index 00000000000..4a7b550cb4a --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/internal/MongodbSingleCollectionProvider.java @@ -0,0 +1,97 @@ +/* + * 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.mongodb.internal; + +import org.bson.BsonDocument; + +import com.google.common.base.Preconditions; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class MongodbSingleCollectionProvider implements MongodbClientProvider { + + private final String connectionString; + + private final String defaultDatabase; + + private final String defaultCollection; + + private MongoClient client; + + private MongoDatabase database; + + private MongoCollection collection; + + public MongodbSingleCollectionProvider( + String connectionString, String defaultDatabase, String defaultCollection) { + Preconditions.checkNotNull(connectionString); + Preconditions.checkNotNull(defaultDatabase); + Preconditions.checkNotNull(defaultCollection); + this.connectionString = connectionString; + this.defaultDatabase = defaultDatabase; + this.defaultCollection = defaultCollection; + } + + @Override + public MongoClient getClient() { + synchronized (this) { + if (client == null) { + client = MongoClients.create(connectionString); + } + } + return client; + } + + @Override + public MongoDatabase getDefaultDatabase() { + synchronized (this) { + if (database == null) { + database = getClient().getDatabase(defaultDatabase); + } + } + return database; + } + + @Override + public MongoCollection getDefaultCollection() { + synchronized (this) { + if (collection == null) { + collection = + getDefaultDatabase().getCollection(defaultCollection, BsonDocument.class); + } + } + return collection; + } + + @Override + public void close() { + try { + if (client != null) { + client.close(); + } + } catch (Exception e) { + log.error("Failed to close Mongo client", e); + } finally { + client = null; + } + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java new file mode 100644 index 00000000000..2ed8ceb7fb2 --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/BsonToRowDataConverters.java @@ -0,0 +1,436 @@ +/* + * 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.mongodb.serde; + +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.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.seatunnel.mongodb.exception.MongodbConnectorException; + +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 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 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.mongodb.config.MongodbConfig.DEFAULT_JSON_WRITER_SETTINGS; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.ENCODE_VALUE_FIELD; + +public class BsonToRowDataConverters implements Serializable { + + private static final long serialVersionUID = 1L; + + @FunctionalInterface + public interface BsonToRowDataConverter extends Serializable { + Object convert(BsonValue bsonValue); + } + + public BsonToRowDataConverter createConverter(SeaTunnelDataType type) { + SerializableFunction internalRowConverter = + createNullSafeInternalConverter(type); + return new BsonToRowDataConverter() { + 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(BsonValue bsonValue) { + return bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN(); + } + + private static SerializableFunction createInternalConverter( + 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(); + } + + private static SerializableFunction createRowConverter( + SeaTunnelRowType type) { + SeaTunnelDataType[] fieldTypes = type.getFieldTypes(); + final SerializableFunction[] fieldConverters = + Arrays.stream(fieldTypes) + .map(BsonToRowDataConverters::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 SerializableFunction createArrayConverter( + 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 SerializableFunction createMapConverter( + String typeSummary, 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(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(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(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(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(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(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(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-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/Serializer.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/DocumentDeserializer.java similarity index 77% rename from seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/Serializer.java rename to seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/DocumentDeserializer.java index f6608244e32..1d75b277a2d 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/Serializer.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/DocumentDeserializer.java @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.mongodb.data; +package org.apache.seatunnel.connectors.seatunnel.mongodb.serde; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -import org.bson.Document; +import org.bson.BsonDocument; import java.io.Serializable; -public interface Serializer extends Serializable { - Document serialize(SeaTunnelRow row); +public interface DocumentDeserializer extends Serializable { + + T deserialize(BsonDocument bsonDocument); } diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/DocumentRowDataDeserializer.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/DocumentRowDataDeserializer.java new file mode 100644 index 00000000000..15a0147ad9f --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/DocumentRowDataDeserializer.java @@ -0,0 +1,75 @@ +/* + * 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.mongodb.serde; + +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.mongodb.exception.MongodbConnectorException; + +import org.bson.BsonDocument; +import org.bson.BsonValue; + +import static org.apache.seatunnel.api.table.type.SqlType.STRING; +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_OPERATION; + +public class DocumentRowDataDeserializer implements DocumentDeserializer { + + private final String[] fieldNames; + + private final SeaTunnelDataType[] fieldTypes; + + private final BsonToRowDataConverters bsonConverters; + + private final Boolean flatSyncString; + + public DocumentRowDataDeserializer( + String[] fieldNames, SeaTunnelDataType dataTypes, Boolean flatSyncString) { + if (fieldNames == null || fieldNames.length < 1) { + throw new MongodbConnectorException(ILLEGAL_ARGUMENT, "fieldName is empty"); + } + this.bsonConverters = new BsonToRowDataConverters(); + this.fieldNames = fieldNames; + this.fieldTypes = ((SeaTunnelRowType) dataTypes).getFieldTypes(); + this.flatSyncString = flatSyncString; + } + + @Override + public SeaTunnelRow deserialize(BsonDocument bsonDocument) { + if (flatSyncString) { + if (fieldNames.length != 1 && fieldTypes[0].getSqlType() != STRING) { + throw new MongodbConnectorException( + UNSUPPORTED_OPERATION, + "By utilizing flatSyncString, only one field attribute value can be set, and the field type must be a String. This operation will perform a string mapping on a single MongoDB data entry."); + } + SeaTunnelRow rowData = new SeaTunnelRow(fieldNames.length); + rowData.setField( + 0, bsonConverters.createConverter(fieldTypes[0]).convert(bsonDocument)); + return rowData; + } + SeaTunnelRow rowData = new SeaTunnelRow(fieldNames.length); + for (int i = 0; i < fieldNames.length; i++) { + String fieldName = this.fieldNames[i]; + BsonValue o = bsonDocument.get(fieldName); + SeaTunnelDataType fieldType = fieldTypes[i]; + rowData.setField(i, bsonConverters.createConverter(fieldType).convert(o)); + } + return rowData; + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/DocumentSerializer.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/DocumentSerializer.java new file mode 100644 index 00000000000..47fd87c5e4b --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/DocumentSerializer.java @@ -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. + */ + +package org.apache.seatunnel.connectors.seatunnel.mongodb.serde; + +import org.bson.BsonDocument; + +import com.mongodb.client.model.WriteModel; + +import java.io.Serializable; + +public interface DocumentSerializer extends Serializable { + + WriteModel serializeToWriteModel(T object); +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/RowDataDocumentSerializer.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/RowDataDocumentSerializer.java new file mode 100644 index 00000000000..6ba99fec5d3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/RowDataDocumentSerializer.java @@ -0,0 +1,74 @@ +/* + * 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.mongodb.serde; + +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.mongodb.sink.MongodbWriterOptions; + +import org.bson.BsonDocument; +import org.bson.conversions.Bson; + +import com.mongodb.client.model.Filters; +import com.mongodb.client.model.InsertOneModel; +import com.mongodb.client.model.UpdateOneModel; +import com.mongodb.client.model.UpdateOptions; +import com.mongodb.client.model.WriteModel; + +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +public class RowDataDocumentSerializer implements DocumentSerializer { + + private final RowDataToBsonConverters.RowDataToBsonConverter rowDataToBsonConverter; + + private final Boolean isUpsertEnable; + + private final Function filterConditions; + + public RowDataDocumentSerializer( + RowDataToBsonConverters.RowDataToBsonConverter rowDataToBsonConverter, + MongodbWriterOptions options, + Function filterConditions) { + this.rowDataToBsonConverter = rowDataToBsonConverter; + this.isUpsertEnable = options.isUpsertEnable(); + this.filterConditions = filterConditions; + } + + @Override + public WriteModel serializeToWriteModel(SeaTunnelRow row) { + final BsonDocument bsonDocument = rowDataToBsonConverter.convert(row); + if (isUpsertEnable) { + Bson filter = generateFilter(filterConditions.apply(bsonDocument)); + bsonDocument.remove("_id"); + BsonDocument update = new BsonDocument("$set", bsonDocument); + return new UpdateOneModel<>(filter, update, new UpdateOptions().upsert(true)); + } else { + return new InsertOneModel<>(bsonDocument); + } + } + + public static Bson generateFilter(BsonDocument filterConditions) { + List filters = + filterConditions.entrySet().stream() + .map(entry -> Filters.eq(entry.getKey(), entry.getValue())) + .collect(Collectors.toList()); + + return Filters.and(filters); + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/RowDataToBsonConverters.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/RowDataToBsonConverters.java new file mode 100644 index 00000000000..a2122e9a3c3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/RowDataToBsonConverters.java @@ -0,0 +1,332 @@ +/* + * 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.mongodb.serde; + +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.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.mongodb.exception.MongodbConnectorException; + +import org.bson.BsonArray; +import org.bson.BsonBinary; +import org.bson.BsonBoolean; +import org.bson.BsonDateTime; +import org.bson.BsonDecimal128; +import org.bson.BsonDocument; +import org.bson.BsonDouble; +import org.bson.BsonInt32; +import org.bson.BsonInt64; +import org.bson.BsonNull; +import org.bson.BsonString; +import org.bson.BsonValue; +import org.bson.json.JsonParseException; +import org.bson.types.Decimal128; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.apache.seatunnel.api.table.type.SqlType.NULL; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_DATA_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.ENCODE_VALUE_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.serde.BsonToRowDataConverters.fromBigDecimal; + +public class RowDataToBsonConverters implements Serializable { + + private static final long serialVersionUID = 1L; + + @FunctionalInterface + public interface RowDataToBsonConverter extends Serializable { + BsonDocument convert(SeaTunnelRow rowData); + } + + public static RowDataToBsonConverter createConverter(SeaTunnelDataType type) { + SerializableFunction internalRowConverter = + createNullSafeInternalConverter(type); + return new RowDataToBsonConverter() { + private static final long serialVersionUID = 1L; + + @Override + public BsonDocument convert(SeaTunnelRow rowData) { + return (BsonDocument) internalRowConverter.apply(rowData); + } + }; + } + + 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 BsonValue apply(Object value) { + if (value == null || NULL.equals(type.getSqlType())) { + throw new MongodbConnectorException( + UNSUPPORTED_DATA_TYPE, + "The column type is <" + + type + + ">, but a null value is being written into it"); + } else { + return internalConverter.apply(value); + } + } + }; + } + + private static SerializableFunction createInternalConverter( + SeaTunnelDataType type) { + switch (type.getSqlType()) { + case NULL: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + return BsonNull.VALUE; + } + }; + case BOOLEAN: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + return new BsonBoolean((boolean) value); + } + }; + case TINYINT: + case SMALLINT: + case INT: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + int intValue = + value instanceof Byte + ? ((Byte) value) & 0xFF + : value instanceof Short + ? ((Short) value).intValue() + : (int) value; + return new BsonInt32(intValue); + } + }; + case BIGINT: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + return new BsonInt64((long) value); + } + }; + case FLOAT: + case DOUBLE: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + double v = + value instanceof Float + ? ((Float) value).doubleValue() + : (double) value; + return new BsonDouble(v); + } + }; + case STRING: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + String val = value.toString(); + // try to parse out the mongodb specific data type from extend-json. + if (val.startsWith("{") + && val.endsWith("}") + && val.contains(ENCODE_VALUE_FIELD)) { + try { + BsonDocument doc = BsonDocument.parse(val); + if (doc.containsKey(ENCODE_VALUE_FIELD)) { + return doc.get(ENCODE_VALUE_FIELD); + } + } catch (JsonParseException e) { + // invalid json format, fallback to store as a bson string. + return new BsonString(value.toString()); + } + } + return new BsonString(value.toString()); + } + }; + case BYTES: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + return new BsonBinary((byte[]) value); + } + }; + case DATE: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + LocalDate localDate = (LocalDate) value; + return new BsonDateTime( + localDate + .atStartOfDay(ZoneId.systemDefault()) + .toInstant() + .toEpochMilli()); + } + }; + case TIMESTAMP: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + LocalDateTime localDateTime = (LocalDateTime) value; + return new BsonDateTime( + localDateTime + .atZone(ZoneId.systemDefault()) + .toInstant() + .toEpochMilli()); + } + }; + case DECIMAL: + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + DecimalType decimalType = (DecimalType) type; + BigDecimal decimalVal = (BigDecimal) value; + return new BsonDecimal128( + new Decimal128( + Objects.requireNonNull( + fromBigDecimal( + decimalVal, + 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 SerializableFunction createArrayConverter( + ArrayType arrayType) { + final SerializableFunction elementConverter = + createNullSafeInternalConverter(arrayType.getElementType()); + + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + Object[] arrayData = (Object[]) value; + final List bsonValues = new ArrayList<>(); + for (Object element : arrayData) { + bsonValues.add(elementConverter.apply(element)); + } + return new BsonArray(bsonValues); + } + }; + } + + private static SerializableFunction createMapConverter( + String typeSummary, SeaTunnelDataType keyType, SeaTunnelDataType valueType) { + if (!SqlType.STRING.equals(keyType.getSqlType())) { + throw new MongodbConnectorException( + CommonErrorCode.UNSUPPORTED_OPERATION, + "JSON format doesn't support non-string as key type of map. The type is: " + + typeSummary); + } + + final SerializableFunction valueConverter = + createNullSafeInternalConverter(valueType); + + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + Map mapData = (Map) value; + final BsonDocument document = new BsonDocument(); + for (Map.Entry entry : mapData.entrySet()) { + String fieldName = entry.getKey(); + document.append(fieldName, valueConverter.apply(entry.getValue())); + } + return document; + } + }; + } + + private static SerializableFunction createRowConverter( + SeaTunnelRowType rowType) { + final SerializableFunction[] fieldConverters = + rowType.getChildren().stream() + .map(RowDataToBsonConverters::createNullSafeInternalConverter) + .toArray(SerializableFunction[]::new); + + final int fieldCount = rowType.getTotalFields(); + final String[] fieldNames = rowType.getFieldNames(); + + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(Object value) { + final SeaTunnelRow rowData = (SeaTunnelRow) value; + final BsonDocument document = new BsonDocument(); + for (int i = 0; i < fieldCount; i++) { + document.append(fieldNames[i], fieldConverters[i].apply(rowData.getField(i))); + } + return document; + } + }; + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/Deserializer.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/SerializableFunction.java similarity index 76% rename from seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/Deserializer.java rename to seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/SerializableFunction.java index e13d1a1acee..12478af4829 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/data/Deserializer.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/serde/SerializableFunction.java @@ -15,14 +15,10 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.mongodb.data; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; - -import org.bson.Document; +package org.apache.seatunnel.connectors.seatunnel.mongodb.serde; import java.io.Serializable; +import java.util.function.Function; -public interface Deserializer extends Serializable { - SeaTunnelRow deserialize(Document document); -} +@FunctionalInterface +public interface SerializableFunction extends Function, Serializable {} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongoKeyExtractor.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongoKeyExtractor.java new file mode 100644 index 00000000000..b258c45d656 --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongoKeyExtractor.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.mongodb.sink; + +import org.apache.seatunnel.connectors.seatunnel.mongodb.serde.SerializableFunction; + +import org.bson.BsonDocument; + +import java.util.Arrays; +import java.util.stream.Collectors; + +public class MongoKeyExtractor implements SerializableFunction { + + private static final long serialVersionUID = 1L; + + private final String[] upsertKey; + + public MongoKeyExtractor(MongodbWriterOptions options) { + upsertKey = options.getUpsertKey(); + } + + @Override + public BsonDocument apply(BsonDocument bsonDocument) { + return Arrays.stream(upsertKey) + .filter(bsonDocument::containsKey) + .collect( + Collectors.toMap( + key -> key, bsonDocument::get, (v1, v2) -> v1, BsonDocument::new)); + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSink.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSink.java index a87dac43ee7..47d39a7f70b 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSink.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSink.java @@ -20,70 +20,95 @@ 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.api.sink.SinkWriter; -import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; 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.config.CheckConfigUtil; -import org.apache.seatunnel.common.config.CheckResult; -import org.apache.seatunnel.common.constants.PluginType; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSimpleSink; import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; import org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig; -import org.apache.seatunnel.connectors.seatunnel.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.mongodb.serde.RowDataDocumentSerializer; +import org.apache.seatunnel.connectors.seatunnel.mongodb.serde.RowDataToBsonConverters; import com.google.auto.service.AutoService; import java.io.IOException; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.COLLECTION; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.DATABASE; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.URI; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.CONNECTOR_IDENTITY; @AutoService(SeaTunnelSink.class) public class MongodbSink extends AbstractSimpleSink { - private SeaTunnelRowType rowType; + private MongodbWriterOptions options; - private MongodbConfig params; + private SeaTunnelRowType seaTunnelRowType; @Override - public String getPluginName() { - return "MongoDB"; + public void prepare(Config pluginConfig) throws PrepareFailException { + if (pluginConfig.hasPath(MongodbConfig.URI.key()) + && pluginConfig.hasPath(MongodbConfig.DATABASE.key()) + && pluginConfig.hasPath(MongodbConfig.COLLECTION.key())) { + String connection = pluginConfig.getString(MongodbConfig.URI.key()); + String database = pluginConfig.getString(MongodbConfig.DATABASE.key()); + String collection = pluginConfig.getString(MongodbConfig.COLLECTION.key()); + MongodbWriterOptions.Builder builder = + MongodbWriterOptions.builder() + .withConnectString(connection) + .withDatabase(database) + .withCollection(collection); + if (pluginConfig.hasPath(MongodbConfig.BUFFER_FLUSH_MAX_ROWS.key())) { + builder.withFlushSize( + pluginConfig.getInt(MongodbConfig.BUFFER_FLUSH_MAX_ROWS.key())); + } + if (pluginConfig.hasPath(MongodbConfig.BUFFER_FLUSH_INTERVAL.key())) { + builder.withBatchIntervalMs( + pluginConfig.getLong(MongodbConfig.BUFFER_FLUSH_INTERVAL.key())); + } + if (pluginConfig.hasPath(MongodbConfig.UPSERT_KEY.key())) { + builder.withUpsertKey( + pluginConfig + .getStringList(MongodbConfig.UPSERT_KEY.key()) + .toArray(new String[0])); + } + if (pluginConfig.hasPath(MongodbConfig.UPSERT_ENABLE.key())) { + builder.withUpsertEnable( + pluginConfig.getBoolean(MongodbConfig.UPSERT_ENABLE.key())); + } + if (pluginConfig.hasPath(MongodbConfig.RETRY_MAX.key())) { + builder.withRetryMax(pluginConfig.getInt(MongodbConfig.RETRY_MAX.key())); + } + if (pluginConfig.hasPath(MongodbConfig.RETRY_INTERVAL.key())) { + builder.withRetryInterval(pluginConfig.getLong(MongodbConfig.RETRY_INTERVAL.key())); + } + this.options = builder.build(); + } } @Override - public void prepare(Config config) throws PrepareFailException { - CheckResult result = - CheckConfigUtil.checkAllExists(config, URI.key(), DATABASE.key(), COLLECTION.key()); - if (!result.isSuccess()) { - throw new MongodbConnectorException( - SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - String.format( - "PluginName: %s, PluginType: %s, Message: %s", - getPluginName(), PluginType.SINK, result.getMsg())); - } - - this.params = MongodbConfig.buildWithConfig(config); + public String getPluginName() { + return CONNECTOR_IDENTITY; } @Override - public void setTypeInfo(SeaTunnelRowType rowType) { - this.rowType = rowType; + public void setTypeInfo(SeaTunnelRowType seaTunnelRowType) { + this.seaTunnelRowType = seaTunnelRowType; } @Override public SeaTunnelDataType getConsumedType() { - return rowType; + return seaTunnelRowType; } @Override public AbstractSinkWriter createWriter(SinkWriter.Context context) throws IOException { - boolean useSimpleTextSchema = CatalogTableUtil.buildSimpleTextSchema().equals(rowType); - return new MongodbSinkWriter(rowType, useSimpleTextSchema, params); + return new MongodbWriter( + new RowDataDocumentSerializer( + RowDataToBsonConverters.createConverter(seaTunnelRowType), + options, + new MongoKeyExtractor(options)), + options, + context); } } diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkFactory.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkFactory.java index c1bfae8944a..2b84d672b9c 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkFactory.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkFactory.java @@ -18,24 +18,37 @@ package org.apache.seatunnel.connectors.seatunnel.mongodb.sink; import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.factory.Factory; import org.apache.seatunnel.api.table.factory.TableSinkFactory; +import org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig; import com.google.auto.service.AutoService; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.COLLECTION; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.DATABASE; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.URI; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.CONNECTOR_IDENTITY; @AutoService(Factory.class) public class MongodbSinkFactory implements TableSinkFactory { @Override public String factoryIdentifier() { - return "MongoDB"; + return CONNECTOR_IDENTITY; } @Override public OptionRule optionRule() { - return OptionRule.builder().required(URI, DATABASE, COLLECTION).build(); + return OptionRule.builder() + .required( + MongodbConfig.URI, + MongodbConfig.DATABASE, + MongodbConfig.COLLECTION, + CatalogTableUtil.SCHEMA) + .optional( + MongodbConfig.BUFFER_FLUSH_INTERVAL, + MongodbConfig.BUFFER_FLUSH_MAX_ROWS, + MongodbConfig.RETRY_MAX, + MongodbConfig.RETRY_INTERVAL, + MongodbConfig.UPSERT_ENABLE, + MongodbConfig.UPSERT_KEY) + .build(); } } diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkWriter.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkWriter.java deleted file mode 100644 index 02446faa88d..00000000000 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbSinkWriter.java +++ /dev/null @@ -1,80 +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.mongodb.sink; - -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; -import org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig; -import org.apache.seatunnel.connectors.seatunnel.mongodb.data.DefaultSerializer; -import org.apache.seatunnel.connectors.seatunnel.mongodb.data.Serializer; - -import org.bson.Document; - -import com.mongodb.client.MongoClient; -import com.mongodb.client.MongoClients; -import com.mongodb.client.MongoCollection; - -import java.io.IOException; - -public class MongodbSinkWriter extends AbstractSinkWriter { - - private final SeaTunnelRowType rowType; - - private final Serializer serializer; - - private final MongoClient client; - - private final String database; - - private final String collection; - - private final MongoCollection mongoCollection; - - private final boolean useSimpleTextSchema; - - public MongodbSinkWriter( - SeaTunnelRowType rowType, boolean useSimpleTextSchema, MongodbConfig params) { - this.rowType = rowType; - this.database = params.getDatabase(); - this.collection = params.getCollection(); - this.client = MongoClients.create(params.getUri()); - this.mongoCollection = this.client.getDatabase(database).getCollection(collection); - this.useSimpleTextSchema = useSimpleTextSchema; - this.serializer = useSimpleTextSchema ? null : new DefaultSerializer(rowType); - } - - @Override - public void write(SeaTunnelRow row) throws IOException { - Document document; - if (useSimpleTextSchema) { - String simpleText = row.getField(0).toString(); - document = Document.parse(simpleText); - } else { - document = serializer.serialize(row); - } - mongoCollection.insertOne(document); - } - - @Override - public void close() throws IOException { - if (client != null) { - client.close(); - } - } -} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbWriter.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbWriter.java new file mode 100644 index 00000000000..299c6c4a63d --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbWriter.java @@ -0,0 +1,167 @@ +/* + * 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.mongodb.sink; + +import org.apache.seatunnel.api.sink.SinkWriter; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.common.sink.AbstractSinkWriter; +import org.apache.seatunnel.connectors.seatunnel.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.mongodb.internal.MongodbClientProvider; +import org.apache.seatunnel.connectors.seatunnel.mongodb.internal.MongodbCollectionProvider; +import org.apache.seatunnel.connectors.seatunnel.mongodb.serde.DocumentSerializer; + +import org.bson.BsonDocument; + +import com.mongodb.MongoException; +import com.mongodb.client.model.BulkWriteOptions; +import com.mongodb.client.model.WriteModel; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.WRITER_OPERATION_FAILED; + +@Slf4j +public class MongodbWriter extends AbstractSinkWriter { + + private MongodbClientProvider collectionProvider; + + private final DocumentSerializer serializer; + + private long bulkActions; + + private final List> bulkRequests; + + private int maxRetries; + + private long retryIntervalMs; + + private long batchIntervalMs; + + private volatile long lastSendTime = 0L; + + private final SinkWriter.Context context; + + public MongodbWriter( + DocumentSerializer serializer, + MongodbWriterOptions options, + SinkWriter.Context context) { + initOptions(options); + this.context = context; + this.serializer = serializer; + this.bulkRequests = new ArrayList<>(); + } + + private void initOptions(MongodbWriterOptions options) { + this.maxRetries = options.getRetryMax(); + this.retryIntervalMs = options.getRetryInterval(); + this.collectionProvider = + MongodbCollectionProvider.builder() + .connectionString(options.getConnectString()) + .database(options.getDatabase()) + .collection(options.getCollection()) + .build(); + this.bulkActions = options.getFlushSize(); + this.batchIntervalMs = options.getBatchIntervalMs(); + } + + @Override + public void write(SeaTunnelRow o) throws IOException { + bulkRequests.add(serializer.serializeToWriteModel(o)); + if (isOverMaxBatchSizeLimit() || isOverMaxBatchIntervalLimit()) { + doBulkWrite(); + } + } + + @Override + public Optional prepareCommit() { + doBulkWrite(); + return Optional.empty(); + } + + @Override + public void close() throws IOException { + doBulkWrite(); + if (collectionProvider != null) { + collectionProvider.close(); + } + } + + synchronized void doBulkWrite() { + if (bulkRequests.isEmpty()) { + // no records to write + return; + } + + boolean success = + IntStream.rangeClosed(0, maxRetries) + .anyMatch( + i -> { + try { + lastSendTime = System.currentTimeMillis(); + collectionProvider + .getDefaultCollection() + .bulkWrite( + bulkRequests, + new BulkWriteOptions().ordered(true)); + bulkRequests.clear(); + return true; + } catch (MongoException e) { + log.debug( + "Bulk Write to MongoDB failed, retry times = {}", + i, + e); + if (i >= maxRetries) { + throw new MongodbConnectorException( + WRITER_OPERATION_FAILED, + "Bulk Write to MongoDB failed", + e); + } + try { + TimeUnit.MILLISECONDS.sleep(retryIntervalMs * (i + 1)); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new MongodbConnectorException( + WRITER_OPERATION_FAILED, + "Unable to flush; interrupted while doing another attempt", + e); + } + return false; + } + }); + + if (!success) { + throw new MongodbConnectorException( + WRITER_OPERATION_FAILED, "Bulk Write to MongoDB failed after max retries"); + } + } + + private boolean isOverMaxBatchSizeLimit() { + return bulkActions != -1 && bulkRequests.size() >= bulkActions; + } + + private boolean isOverMaxBatchIntervalLimit() { + long lastSentInterval = System.currentTimeMillis() - lastSendTime; + return batchIntervalMs != -1 && lastSentInterval >= batchIntervalMs; + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbWriterOptions.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbWriterOptions.java new file mode 100644 index 00000000000..e37a4d22cda --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/sink/MongodbWriterOptions.java @@ -0,0 +1,150 @@ +/* + * 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.mongodb.sink; + +import lombok.Getter; + +import java.io.Serializable; + +@Getter +public class MongodbWriterOptions implements Serializable { + + private static final long serialVersionUID = 1; + + protected final String connectString; + + protected final String database; + + protected final String collection; + + protected final int flushSize; + + protected final long batchIntervalMs; + + protected final boolean upsertEnable; + + protected final String[] upsertKey; + + protected final int retryMax; + + protected final long retryInterval; + + public MongodbWriterOptions( + String connectString, + String database, + String collection, + int flushSize, + Long batchIntervalMs, + boolean upsertEnable, + String[] upsertKey, + int retryMax, + Long retryInterval) { + this.connectString = connectString; + this.database = database; + this.collection = collection; + this.flushSize = flushSize; + this.batchIntervalMs = batchIntervalMs; + this.upsertEnable = upsertEnable; + this.upsertKey = upsertKey; + this.retryMax = retryMax; + this.retryInterval = retryInterval; + } + + public static Builder builder() { + return new Builder(); + } + + /** Builder For {@link MongodbWriterOptions}. */ + public static class Builder { + protected String connectString; + + protected String database; + + protected String collection; + + protected int flushSize; + + protected long batchIntervalMs; + + protected boolean upsertEnable; + + protected String[] upsertKey; + + protected int retryMax; + + protected long retryInterval; + + public Builder withConnectString(String connectString) { + this.connectString = connectString; + return this; + } + + public Builder withDatabase(String database) { + this.database = database; + return this; + } + + public Builder withCollection(String collection) { + this.collection = collection; + return this; + } + + public Builder withFlushSize(int flushSize) { + this.flushSize = flushSize; + return this; + } + + public Builder withBatchIntervalMs(Long batchIntervalMs) { + this.batchIntervalMs = batchIntervalMs; + return this; + } + + public Builder withUpsertEnable(boolean upsertEnable) { + this.upsertEnable = upsertEnable; + return this; + } + + public Builder withUpsertKey(String[] upsertKey) { + this.upsertKey = upsertKey; + return this; + } + + public Builder withRetryMax(int retryMax) { + this.retryMax = retryMax; + return this; + } + + public Builder withRetryInterval(Long retryInterval) { + this.retryInterval = retryInterval; + return this; + } + + public MongodbWriterOptions build() { + return new MongodbWriterOptions( + connectString, + database, + collection, + flushSize, + batchIntervalMs, + upsertEnable, + upsertKey, + retryMax, + retryInterval); + } + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java index 2b210483e58..4165a76b3ae 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSource.java @@ -20,57 +20,125 @@ 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.Boundedness; import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.api.source.SupportColumnProjection; import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; 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.config.CheckConfigUtil; -import org.apache.seatunnel.common.config.CheckResult; -import org.apache.seatunnel.common.constants.PluginType; -import org.apache.seatunnel.connectors.seatunnel.common.source.AbstractSingleSplitReader; -import org.apache.seatunnel.connectors.seatunnel.common.source.AbstractSingleSplitSource; -import org.apache.seatunnel.connectors.seatunnel.common.source.SingleSplitReaderContext; import org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig; -import org.apache.seatunnel.connectors.seatunnel.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.mongodb.internal.MongodbClientProvider; +import org.apache.seatunnel.connectors.seatunnel.mongodb.internal.MongodbCollectionProvider; +import org.apache.seatunnel.connectors.seatunnel.mongodb.serde.DocumentDeserializer; +import org.apache.seatunnel.connectors.seatunnel.mongodb.serde.DocumentRowDataDeserializer; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.config.MongodbReadOptions; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.enumerator.MongodbSplitEnumerator; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.reader.MongodbReader; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.split.MongoSplit; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.split.MongoSplitStrategy; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.split.SamplingSplitStrategy; + +import org.bson.BsonDocument; import com.google.auto.service.AutoService; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.COLLECTION; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.DATABASE; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.URI; +import java.util.ArrayList; + +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.CONNECTOR_IDENTITY; @AutoService(SeaTunnelSource.class) -public class MongodbSource extends AbstractSingleSplitSource { +public class MongodbSource + implements SeaTunnelSource>, + SupportColumnProjection { + + private static final long serialVersionUID = 1L; + + private MongodbClientProvider clientProvider; + + private DocumentDeserializer deserializer; + + private MongoSplitStrategy splitStrategy; private SeaTunnelRowType rowType; - private MongodbConfig params; + private MongodbReadOptions mongodbReadOptions; @Override public String getPluginName() { - return "MongoDB"; + return CONNECTOR_IDENTITY; } @Override - public void prepare(Config config) throws PrepareFailException { - CheckResult result = - CheckConfigUtil.checkAllExists(config, URI.key(), DATABASE.key(), COLLECTION.key()); - if (!result.isSuccess()) { - throw new MongodbConnectorException( - SeaTunnelAPIErrorCode.CONFIG_VALIDATION_FAILED, - String.format( - "PluginName: %s, PluginType: %s, Message: %s", - getPluginName(), PluginType.SOURCE, result.getMsg())); + public void prepare(Config pluginConfig) throws PrepareFailException { + if (pluginConfig.hasPath(MongodbConfig.URI.key()) + && pluginConfig.hasPath(MongodbConfig.DATABASE.key()) + && pluginConfig.hasPath(MongodbConfig.COLLECTION.key())) { + String connection = pluginConfig.getString(MongodbConfig.URI.key()); + String database = pluginConfig.getString(MongodbConfig.DATABASE.key()); + String collection = pluginConfig.getString(MongodbConfig.COLLECTION.key()); + clientProvider = + MongodbCollectionProvider.builder() + .connectionString(connection) + .database(database) + .collection(collection) + .build(); } - this.params = MongodbConfig.buildWithConfig(config); - if (config.hasPath(CatalogTableUtil.SCHEMA.key())) { - this.rowType = CatalogTableUtil.buildWithConfig(config).getSeaTunnelRowType(); + if (pluginConfig.hasPath(CatalogTableUtil.SCHEMA.key())) { + this.rowType = CatalogTableUtil.buildWithConfig(pluginConfig).getSeaTunnelRowType(); } else { this.rowType = CatalogTableUtil.buildSimpleTextSchema(); } + + if (pluginConfig.hasPath(MongodbConfig.FLAT_SYNC_STRING.key())) { + deserializer = + new DocumentRowDataDeserializer( + rowType.getFieldNames(), + rowType, + pluginConfig.getBoolean(MongodbConfig.FLAT_SYNC_STRING.key())); + } else { + deserializer = + new DocumentRowDataDeserializer( + rowType.getFieldNames(), + rowType, + MongodbConfig.FLAT_SYNC_STRING.defaultValue()); + } + + SamplingSplitStrategy.Builder splitStrategyBuilder = SamplingSplitStrategy.builder(); + if (pluginConfig.hasPath(MongodbConfig.MATCH_QUERY.key())) { + splitStrategyBuilder.setMatchQuery( + BsonDocument.parse(pluginConfig.getString(MongodbConfig.MATCH_QUERY.key()))); + } + if (pluginConfig.hasPath(MongodbConfig.SPLIT_KEY.key())) { + splitStrategyBuilder.setSplitKey(pluginConfig.getString(MongodbConfig.SPLIT_KEY.key())); + } + if (pluginConfig.hasPath(MongodbConfig.SPLIT_SIZE.key())) { + splitStrategyBuilder.setSizePerSplit( + pluginConfig.getLong(MongodbConfig.SPLIT_SIZE.key())); + } + if (pluginConfig.hasPath(MongodbConfig.PROJECTION.key())) { + splitStrategyBuilder.setProjection( + BsonDocument.parse(pluginConfig.getString(MongodbConfig.PROJECTION.key()))); + } + splitStrategy = splitStrategyBuilder.setClientProvider(clientProvider).build(); + + MongodbReadOptions.MongoReadOptionsBuilder mongoReadOptionsBuilder = + MongodbReadOptions.builder(); + if (pluginConfig.hasPath(MongodbConfig.MAX_TIME_MIN.key())) { + mongoReadOptionsBuilder.setMaxTimeMS( + pluginConfig.getLong(MongodbConfig.MAX_TIME_MIN.key())); + } + if (pluginConfig.hasPath(MongodbConfig.FETCH_SIZE.key())) { + mongoReadOptionsBuilder.setFetchSize( + pluginConfig.getInt(MongodbConfig.FETCH_SIZE.key())); + } + if (pluginConfig.hasPath(MongodbConfig.CURSOR_NO_TIMEOUT.key())) { + mongoReadOptionsBuilder.setNoCursorTimeout( + pluginConfig.getBoolean(MongodbConfig.CURSOR_NO_TIMEOUT.key())); + } + mongodbReadOptions = mongoReadOptionsBuilder.build(); } @Override @@ -80,13 +148,27 @@ public Boundedness getBoundedness() { @Override public SeaTunnelDataType getProducedType() { - return this.rowType; + return rowType; + } + + @Override + public SourceReader createReader(SourceReader.Context readerContext) + throws Exception { + return new MongodbReader(readerContext, clientProvider, deserializer, mongodbReadOptions); + } + + @Override + public SourceSplitEnumerator> createEnumerator( + SourceSplitEnumerator.Context enumeratorContext) throws Exception { + return new MongodbSplitEnumerator(enumeratorContext, clientProvider, splitStrategy); } @Override - public AbstractSingleSplitReader createReader(SingleSplitReaderContext context) + public SourceSplitEnumerator> restoreEnumerator( + SourceSplitEnumerator.Context enumeratorContext, + ArrayList checkpointState) throws Exception { - boolean useSimpleTextSchema = CatalogTableUtil.buildSimpleTextSchema().equals(rowType); - return new MongodbSourceReader(context, this.params, rowType, useSimpleTextSchema); + return new MongodbSplitEnumerator( + enumeratorContext, clientProvider, splitStrategy, checkpointState); } } diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceFactory.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceFactory.java index 7d05cbc8c28..22687c1bf72 100644 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceFactory.java +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceFactory.java @@ -22,31 +22,45 @@ 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.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.split.MongoSplit; import com.google.auto.service.AutoService; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.COLLECTION; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.DATABASE; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.MATCHQUERY; -import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbOption.URI; +import java.util.ArrayList; + +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.CONNECTOR_IDENTITY; @AutoService(Factory.class) public class MongodbSourceFactory implements TableSourceFactory { @Override public String factoryIdentifier() { - return "MongoDB"; + return CONNECTOR_IDENTITY; } @Override public OptionRule optionRule() { return OptionRule.builder() - .required(URI, DATABASE, COLLECTION, CatalogTableUtil.SCHEMA) - .optional(MATCHQUERY) + .required( + MongodbConfig.URI, + MongodbConfig.DATABASE, + MongodbConfig.COLLECTION, + CatalogTableUtil.SCHEMA) + .optional( + MongodbConfig.PROJECTION, + MongodbConfig.MATCH_QUERY, + MongodbConfig.SPLIT_SIZE, + MongodbConfig.SPLIT_KEY, + MongodbConfig.CURSOR_NO_TIMEOUT, + MongodbConfig.FETCH_SIZE, + MongodbConfig.MAX_TIME_MIN) .build(); } @Override - public Class getSourceClass() { + public Class>> + getSourceClass() { return MongodbSource.class; } } diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceReader.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceReader.java deleted file mode 100644 index 4fbca78ff60..00000000000 --- a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/MongodbSourceReader.java +++ /dev/null @@ -1,118 +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.mongodb.source; - -import org.apache.seatunnel.api.source.Boundedness; -import org.apache.seatunnel.api.source.Collector; -import org.apache.seatunnel.api.source.SupportColumnProjection; -import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; -import org.apache.seatunnel.connectors.seatunnel.common.source.AbstractSingleSplitReader; -import org.apache.seatunnel.connectors.seatunnel.common.source.SingleSplitReaderContext; -import org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig; -import org.apache.seatunnel.connectors.seatunnel.mongodb.data.DefaultDeserializer; -import org.apache.seatunnel.connectors.seatunnel.mongodb.data.Deserializer; - -import org.bson.BsonDocument; -import org.bson.Document; -import org.bson.conversions.Bson; - -import com.mongodb.client.MongoClient; -import com.mongodb.client.MongoClients; -import com.mongodb.client.MongoCursor; -import com.mongodb.client.model.Projections; -import lombok.extern.slf4j.Slf4j; - -import java.io.IOException; -import java.util.Optional; - -@Slf4j -public class MongodbSourceReader extends AbstractSingleSplitReader - implements SupportColumnProjection { - - private final SingleSplitReaderContext context; - - private MongoClient client; - - private final MongodbConfig params; - - private final Deserializer deserializer; - - private final Bson projectionFields; - - private final boolean useSimpleTextSchema; - - MongodbSourceReader( - SingleSplitReaderContext context, - MongodbConfig params, - SeaTunnelRowType rowType, - boolean useSimpleTextSchema) { - this.context = context; - this.params = params; - this.useSimpleTextSchema = useSimpleTextSchema; - if (useSimpleTextSchema) { - this.deserializer = null; - this.projectionFields = null; - } else { - this.deserializer = new DefaultDeserializer(rowType); - this.projectionFields = - Projections.fields( - Projections.include(rowType.getFieldNames()), Projections.excludeId()); - } - } - - @Override - public void open() throws Exception { - client = MongoClients.create(params.getUri()); - } - - @Override - public void close() throws IOException { - if (client != null) { - client.close(); - } - } - - @Override - public void pollNext(Collector output) throws Exception { - try (MongoCursor mongoCursor = - client.getDatabase(params.getDatabase()) - .getCollection(params.getCollection()) - .find( - Optional.ofNullable(params.getMatchQuery()).isPresent() - ? BsonDocument.parse(params.getMatchQuery()) - : new BsonDocument()) - .projection(projectionFields) - .iterator()) { - while (mongoCursor.hasNext()) { - Document document = mongoCursor.next(); - if (useSimpleTextSchema) { - output.collect(new SeaTunnelRow(new Object[] {document.toJson()})); - } else { - output.collect(deserializer.deserialize(document)); - } - } - } finally { - if (Boundedness.BOUNDED.equals(context.getBoundedness())) { - // signal to the source that we have reached the end of the data. - log.info("Closed the bounded mongodb source"); - context.signalNoMoreElement(); - } - } - } -} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/config/MongodbReadOptions.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/config/MongodbReadOptions.java new file mode 100644 index 00000000000..f0020eb41ad --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/config/MongodbReadOptions.java @@ -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. + */ + +package org.apache.seatunnel.connectors.seatunnel.mongodb.source.config; + +import lombok.EqualsAndHashCode; +import lombok.Getter; + +import java.io.Serializable; + +import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.CURSOR_NO_TIMEOUT; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.FETCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.mongodb.config.MongodbConfig.MAX_TIME_MIN; + +/** The configuration class for MongoDB source. */ +@EqualsAndHashCode +@Getter +public class MongodbReadOptions implements Serializable { + + private static final long serialVersionUID = 1L; + + private final int fetchSize; + + private final boolean noCursorTimeout; + + private final long maxTimeMS; + + private MongodbReadOptions(int fetchSize, boolean noCursorTimeout, long maxTimeMS) { + this.fetchSize = fetchSize; + this.noCursorTimeout = noCursorTimeout; + this.maxTimeMS = maxTimeMS; + } + + public static MongoReadOptionsBuilder builder() { + return new MongoReadOptionsBuilder(); + } + + /** Builder for {@link MongodbReadOptions}. */ + public static class MongoReadOptionsBuilder { + + private int fetchSize = FETCH_SIZE.defaultValue(); + + private boolean noCursorTimeout = CURSOR_NO_TIMEOUT.defaultValue(); + + private long maxTimeMin = MAX_TIME_MIN.defaultValue(); + + private MongoReadOptionsBuilder() {} + + public MongoReadOptionsBuilder setFetchSize(int fetchSize) { + checkArgument(fetchSize > 0, "The fetch size must be larger than 0."); + this.fetchSize = fetchSize; + return this; + } + + public MongoReadOptionsBuilder setNoCursorTimeout(boolean noCursorTimeout) { + this.noCursorTimeout = noCursorTimeout; + return this; + } + + public MongoReadOptionsBuilder setMaxTimeMS(long maxTimeMS) { + this.maxTimeMin = maxTimeMS; + return this; + } + + public MongodbReadOptions build() { + return new MongodbReadOptions(fetchSize, noCursorTimeout, maxTimeMin); + } + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/enumerator/MongodbSplitEnumerator.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/enumerator/MongodbSplitEnumerator.java new file mode 100644 index 00000000000..466e853da42 --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/enumerator/MongodbSplitEnumerator.java @@ -0,0 +1,167 @@ +/* + * 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.mongodb.source.enumerator; + +import org.apache.seatunnel.shade.com.google.common.collect.Lists; + +import org.apache.seatunnel.api.source.SourceSplitEnumerator; +import org.apache.seatunnel.common.exception.CommonErrorCode; +import org.apache.seatunnel.connectors.seatunnel.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.mongodb.internal.MongodbClientProvider; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.split.MongoSplit; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.split.MongoSplitStrategy; + +import com.mongodb.MongoNamespace; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** MongoSplitEnumerator generates {@link MongoSplit} according to partition strategies. */ +@Slf4j +public class MongodbSplitEnumerator + implements SourceSplitEnumerator> { + + private final ArrayList pendingSplits = Lists.newArrayList(); + + private final Context context; + + private final MongodbClientProvider clientProvider; + + private final MongoSplitStrategy strategy; + + public MongodbSplitEnumerator( + Context context, + MongodbClientProvider clientProvider, + MongoSplitStrategy strategy) { + this(context, clientProvider, strategy, Collections.emptyList()); + } + + public MongodbSplitEnumerator( + Context context, + MongodbClientProvider clientProvider, + MongoSplitStrategy strategy, + List splits) { + this.context = context; + this.clientProvider = clientProvider; + this.strategy = strategy; + this.pendingSplits.addAll(splits); + } + + @Override + public void open() {} + + @Override + public synchronized void run() throws Exception { + log.info("Starting MongoSplitEnumerator."); + Set readers = context.registeredReaders(); + pendingSplits.addAll(strategy.split()); + MongoNamespace namespace = clientProvider.getDefaultCollection().getNamespace(); + log.info( + "Added {} pending splits for namespace {}.", + pendingSplits.size(), + namespace.getFullName()); + assignSplits(readers); + } + + @Override + public void close() throws IOException { + if (clientProvider != null) { + clientProvider.close(); + } + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + if (splits != null) { + log.info("Received {} split(s) back from subtask {}.", splits.size(), subtaskId); + pendingSplits.addAll(splits); + } + } + + @Override + public int currentUnassignedSplitSize() { + return pendingSplits.size(); + } + + @Override + public void handleSplitRequest(int subtaskId) { + throw new MongodbConnectorException( + CommonErrorCode.UNSUPPORTED_OPERATION, + String.format("Unsupported handleSplitRequest: %d", subtaskId)); + } + + @Override + public void registerReader(int subtaskId) { + log.debug("Register reader {} to MongodbSplitEnumerator.", subtaskId); + if (!pendingSplits.isEmpty()) { + assignSplits(Collections.singletonList(subtaskId)); + } + } + + @Override + public ArrayList snapshotState(long checkpointId) throws Exception { + return pendingSplits; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + // Do nothing + } + + private synchronized void assignSplits(Collection readers) { + log.debug("Assign pendingSplits to readers {}", readers); + int numReaders = readers.size(); + + Map> splitsBySubtaskId = + pendingSplits.stream() + .collect( + Collectors.groupingBy( + split -> getSplitOwner(split.splitId(), numReaders))); + + readers.forEach(subtaskId -> assignSplitsToSubtask(subtaskId, splitsBySubtaskId)); + + pendingSplits.clear(); + readers.forEach(context::signalNoMoreSplits); + } + + private void assignSplitsToSubtask( + Integer subtaskId, Map> splitsBySubtaskId) { + log.info("Received split request from taskId {}.", subtaskId); + + List assignedSplits = + splitsBySubtaskId.getOrDefault(subtaskId, Collections.emptyList()); + + context.assignSplit(subtaskId, assignedSplits); + log.info( + "Assigned {} splits to subtask {}, remaining splits: {}.", + assignedSplits.size(), + subtaskId, + pendingSplits.size() - assignedSplits.size()); + } + + private static int getSplitOwner(String tp, int numReaders) { + return (tp.hashCode() & Integer.MAX_VALUE) % numReaders; + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/reader/MongodbReader.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/reader/MongodbReader.java new file mode 100644 index 00000000000..464aa1b7835 --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/reader/MongodbReader.java @@ -0,0 +1,145 @@ +/* + * 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.mongodb.source.reader; + +import org.apache.seatunnel.shade.com.google.common.base.Preconditions; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.source.SourceReader; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.seatunnel.mongodb.internal.MongodbClientProvider; +import org.apache.seatunnel.connectors.seatunnel.mongodb.serde.DocumentDeserializer; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.config.MongodbReadOptions; +import org.apache.seatunnel.connectors.seatunnel.mongodb.source.split.MongoSplit; + +import org.bson.BsonDocument; + +import com.mongodb.client.FindIterable; +import com.mongodb.client.MongoCursor; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.TimeUnit; + +/** MongoReader reads MongoDB by splits (queries). */ +@Slf4j +public class MongodbReader implements SourceReader { + + private final Queue pendingSplits; + + private final DocumentDeserializer deserializer; + + private final SourceReader.Context context; + + private final MongodbClientProvider clientProvider; + + private MongoCursor cursor; + + private final MongodbReadOptions readOptions; + + private volatile boolean noMoreSplit; + + public MongodbReader( + SourceReader.Context context, + MongodbClientProvider clientProvider, + DocumentDeserializer deserializer, + MongodbReadOptions mongodbReadOptions) { + this.deserializer = deserializer; + this.context = context; + this.clientProvider = clientProvider; + pendingSplits = new ConcurrentLinkedDeque<>(); + this.readOptions = mongodbReadOptions; + } + + @Override + public void open() throws Exception { + if (cursor != null) { + cursor.close(); + } + } + + @Override + public void close() throws IOException { + if (cursor != null) { + cursor.close(); + } + } + + @Override + public void pollNext(Collector output) throws Exception { + synchronized (output.getCheckpointLock()) { + MongoSplit currentSplit = pendingSplits.poll(); + if (null != currentSplit) { + if (cursor != null) { + // current split is in-progress + return; + } + log.info("Prepared to read split {}", currentSplit.splitId()); + FindIterable rs = + clientProvider + .getDefaultCollection() + .find(currentSplit.getQuery()) + .projection(currentSplit.getProjection()) + .batchSize(readOptions.getFetchSize()) + .noCursorTimeout(readOptions.isNoCursorTimeout()) + .maxTime(readOptions.getMaxTimeMS(), TimeUnit.MINUTES); + cursor = rs.iterator(); + while (cursor.hasNext()) { + SeaTunnelRow deserialize = deserializer.deserialize(cursor.next()); + output.collect(deserialize); + } + closeCurrentSplit(); + } + if (noMoreSplit && pendingSplits.isEmpty()) { + // signal to the source that we have reached the end of the data. + log.info("Closed the bounded mongodb source"); + context.signalNoMoreElement(); + } + } + } + + @Override + public List snapshotState(long checkpointId) throws Exception { + return new ArrayList<>(pendingSplits); + } + + @Override + public void addSplits(List splits) { + log.info("Adding split(s) to reader: {}", splits); + pendingSplits.addAll(splits); + } + + @Override + public void handleNoMoreSplits() { + log.info("receive no more splits message, this reader will not add new split."); + noMoreSplit = true; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception {} + + private void closeCurrentSplit() { + Preconditions.checkNotNull(cursor); + cursor.close(); + cursor = null; + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/MongoSplit.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/MongoSplit.java new file mode 100644 index 00000000000..ba029f1dcb9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/MongoSplit.java @@ -0,0 +1,44 @@ +/* + * 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.mongodb.source.split; + +import org.apache.seatunnel.api.source.SourceSplit; + +import org.bson.BsonDocument; + +import lombok.AllArgsConstructor; +import lombok.Getter; + +/** MongoSplit is composed a query and a start offset. */ +@Getter +@AllArgsConstructor +public class MongoSplit implements SourceSplit { + + private final String splitId; + + private final BsonDocument query; + + private final BsonDocument projection; + + private final long startOffset; + + @Override + public String splitId() { + return splitId; + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/MongoSplitStrategy.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/MongoSplitStrategy.java new file mode 100644 index 00000000000..a749085d945 --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/MongoSplitStrategy.java @@ -0,0 +1,26 @@ +/* + * 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.mongodb.source.split; + +import java.util.List; + +/** MongoSplitStrategy defines how to partition a Mongo data set into {@link MongoSplit}s. */ +public interface MongoSplitStrategy { + + List split(); +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/MongoSplitUtils.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/MongoSplitUtils.java new file mode 100644 index 00000000000..45f15d93dac --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/MongoSplitUtils.java @@ -0,0 +1,72 @@ +/* + * 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.mongodb.source.split; + +import org.bson.BsonDocument; + +import javax.annotation.Nullable; + +import static com.mongodb.client.model.Filters.and; +import static com.mongodb.client.model.Filters.gte; +import static com.mongodb.client.model.Filters.lt; + +/** Helper class for using {@link MongoSplit}. */ +public class MongoSplitUtils { + + private static final String SPLIT_ID_TEMPLATE = "split-%d"; + + public static MongoSplit createMongoSplit( + int index, + BsonDocument matchQuery, + BsonDocument projection, + String splitKey, + @Nullable Object lowerBound, + @Nullable Object upperBound) { + return createMongoSplit(index, matchQuery, projection, splitKey, lowerBound, upperBound, 0); + } + + public static MongoSplit createMongoSplit( + int index, + BsonDocument matchQuery, + BsonDocument projection, + String splitKey, + @Nullable Object lowerBound, + @Nullable Object upperBound, + long startOffset) { + BsonDocument splitQuery = new BsonDocument(); + if (matchQuery != null) { + matchQuery.forEach(splitQuery::append); + } + if (splitKey != null) { + BsonDocument boundaryQuery; + if (lowerBound != null && upperBound != null) { + boundaryQuery = + and(gte(splitKey, lowerBound), lt(splitKey, upperBound)).toBsonDocument(); + } else if (lowerBound != null) { + boundaryQuery = gte(splitKey, lowerBound).toBsonDocument(); + } else if (upperBound != null) { + boundaryQuery = lt(splitKey, upperBound).toBsonDocument(); + } else { + boundaryQuery = new BsonDocument(); + } + boundaryQuery.forEach(splitQuery::append); + } + return new MongoSplit( + String.format(SPLIT_ID_TEMPLATE, index), splitQuery, projection, startOffset); + } +} diff --git a/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java new file mode 100644 index 00000000000..44fc116934c --- /dev/null +++ b/seatunnel-connectors-v2/connector-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/mongodb/source/split/SamplingSplitStrategy.java @@ -0,0 +1,252 @@ +/* + * 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.mongodb.source.split; + +import org.apache.seatunnel.shade.com.google.common.base.Preconditions; +import org.apache.seatunnel.shade.com.google.common.collect.Lists; + +import org.apache.seatunnel.connectors.seatunnel.mongodb.internal.MongodbClientProvider; + +import org.apache.commons.lang3.tuple.ImmutablePair; + +import org.bson.BsonDocument; +import org.bson.BsonString; +import org.bson.Document; + +import com.mongodb.client.model.Aggregates; +import com.mongodb.client.model.Projections; +import com.mongodb.client.model.Sorts; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class SamplingSplitStrategy implements MongoSplitStrategy, Serializable { + + private final MongodbClientProvider clientProvider; + + private final String splitKey; + + private final BsonDocument matchQuery; + + private final BsonDocument projection; + + private final long samplesPerSplit; + + private final long sizePerSplit; + + SamplingSplitStrategy( + MongodbClientProvider clientProvider, + String splitKey, + BsonDocument matchQuery, + BsonDocument projection, + long samplesPerSplit, + long sizePerSplit) { + this.clientProvider = clientProvider; + this.splitKey = splitKey; + this.matchQuery = matchQuery; + this.projection = projection; + this.samplesPerSplit = samplesPerSplit; + this.sizePerSplit = sizePerSplit; + } + + @Override + public List split() { + ImmutablePair numAndAvgSize = getDocumentNumAndAvgSize(); + long count = numAndAvgSize.getLeft(); + long avgSize = numAndAvgSize.getRight(); + + long numDocumentsPerSplit = sizePerSplit / avgSize; + int numSplits = (int) Math.ceil((double) count / numDocumentsPerSplit); + int numSamples = (int) Math.floor(samplesPerSplit * numSplits); + + if (numSplits == 0) { + return Lists.newArrayList(); + } + if (numSplits == 1) { + return Lists.newArrayList( + MongoSplitUtils.createMongoSplit( + 0, matchQuery, projection, splitKey, null, null)); + } + List samples = sampleCollection(numSamples); + if (samples.isEmpty()) { + return Collections.emptyList(); + } + + List rightBoundaries = + IntStream.range(0, samples.size()) + .filter( + i -> + i % samplesPerSplit == 0 + || !matchQuery.isEmpty() && i == count - 1) + .mapToObj(i -> samples.get(i).get(splitKey)) + .collect(Collectors.toList()); + + return createSplits(splitKey, rightBoundaries); + } + + private ImmutablePair getDocumentNumAndAvgSize() { + String collectionName = + clientProvider.getDefaultCollection().getNamespace().getCollectionName(); + BsonDocument statsCmd = new BsonDocument("collStats", new BsonString(collectionName)); + Document res = clientProvider.getDefaultDatabase().runCommand(statsCmd); + long total = res.getInteger("count"); + Object avgDocumentBytes = res.get("avgObjSize"); + long avgObjSize = + Optional.ofNullable(avgDocumentBytes) + .map( + docBytes -> { + if (docBytes instanceof Integer) { + return ((Integer) docBytes).longValue(); + } else if (docBytes instanceof Double) { + return ((Double) docBytes).longValue(); + } else { + return 0L; + } + }) + .orElse(0L); + + if (matchQuery == null || matchQuery.isEmpty()) { + return ImmutablePair.of(total, avgObjSize); + } else { + return ImmutablePair.of( + clientProvider.getDefaultCollection().countDocuments(matchQuery), avgObjSize); + } + } + + private List sampleCollection(int numSamples) { + return clientProvider + .getDefaultCollection() + .aggregate( + Lists.newArrayList( + Aggregates.match(matchQuery), + Aggregates.sample(numSamples), + Aggregates.project(Projections.include(splitKey)), + Aggregates.sort(Sorts.ascending(splitKey)))) + .allowDiskUse(true) + .into(Lists.newArrayList()); + } + + private List createSplits(String splitKey, List rightBoundaries) { + if (rightBoundaries.size() == 0) { + return Collections.emptyList(); + } + + List splits = + IntStream.range(0, rightBoundaries.size()) + .mapToObj( + index -> { + Object min = index > 0 ? rightBoundaries.get(index - 1) : null; + return MongoSplitUtils.createMongoSplit( + index, + matchQuery, + projection, + splitKey, + min, + rightBoundaries.get(index)); + }) + .collect(Collectors.toList()); + + Object lastBoundary = rightBoundaries.get(rightBoundaries.size() - 1); + splits.add( + MongoSplitUtils.createMongoSplit( + splits.size(), matchQuery, projection, splitKey, lastBoundary, null)); + return splits; + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private MongodbClientProvider clientProvider; + + private String splitKey; + + private BsonDocument matchQuery; + + private BsonDocument projection; + + private long samplesPerSplit; + + private long sizePerSplit; + + private static final BsonDocument EMPTY_MATCH_QUERY = new BsonDocument(); + + private static final BsonDocument EMPTY_PROJECTION = new BsonDocument(); + + private static final String DEFAULT_SPLIT_KEY = "_id"; + + private static final long DEFAULT_SAMPLES_PER_SPLIT = 10; + + private static final long DEFAULT_SIZE_PER_SPLIT = 64 * 1024 * 1024; + + Builder() { + this.clientProvider = null; + this.splitKey = DEFAULT_SPLIT_KEY; + this.matchQuery = EMPTY_MATCH_QUERY; + this.projection = EMPTY_PROJECTION; + this.samplesPerSplit = DEFAULT_SAMPLES_PER_SPLIT; + this.sizePerSplit = DEFAULT_SIZE_PER_SPLIT; + } + + public Builder setClientProvider(MongodbClientProvider clientProvider) { + this.clientProvider = clientProvider; + return this; + } + + public Builder setSplitKey(String splitKey) { + this.splitKey = splitKey; + return this; + } + + public Builder setMatchQuery(BsonDocument matchQuery) { + this.matchQuery = matchQuery; + return this; + } + + public Builder setProjection(BsonDocument projection) { + this.projection = projection; + return this; + } + + public Builder setSamplesPerSplit(long samplesPerSplit) { + this.samplesPerSplit = samplesPerSplit; + return this; + } + + public Builder setSizePerSplit(long sizePerSplit) { + this.sizePerSplit = sizePerSplit; + return this; + } + + public SamplingSplitStrategy build() { + Preconditions.checkNotNull(clientProvider); + return new SamplingSplitStrategy( + clientProvider, + splitKey, + matchQuery, + projection, + samplesPerSplit, + sizePerSplit); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbIT.java index 80a062e6127..245b709e347 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbIT.java @@ -17,24 +17,12 @@ package org.apache.seatunnel.e2e.connector.v2.mongodb; -import org.apache.seatunnel.api.table.type.ArrayType; -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.MapType; -import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; -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.mongodb.data.DefaultSerializer; -import org.apache.seatunnel.connectors.seatunnel.mongodb.data.Serializer; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.apache.seatunnel.e2e.common.container.TestContainer; import org.awaitility.Awaitility; 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; @@ -49,16 +37,19 @@ 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.io.IOException; -import java.math.BigDecimal; import java.time.Duration; -import java.time.LocalDate; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Random; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import java.util.stream.Stream; import static java.net.HttpURLConnection.HTTP_OK; @@ -67,36 +58,145 @@ @Slf4j public class MongodbIT extends TestSuiteBase implements TestResource { - private static final String MONGODB_IMAGE = "mongo:6.0.5"; + private static final Random RANDOM = new Random(); + + private static final List TEST_MATCH_DATASET = generateTestDataSet(5); + + private static final List TEST_SPLIT_DATASET = generateTestDataSet(10); + + private static final String MONGODB_IMAGE = "mongo:latest"; + private static final String MONGODB_CONTAINER_HOST = "e2e_mongodb"; + private static final int MONGODB_PORT = 27017; + private static final String MONGODB_DATABASE = "test_db"; - private static final String MONGODB_SOURCE_TABLE = "source_table"; - private static final List TEST_DATASET = generateTestDataSet(0, 10); + private static final String MONGODB_MATCH_TABLE = "test_match_op_db"; + + private static final String MONGODB_SPLIT_TABLE = "test_split_op_db"; + + private static final String MONGODB_MATCH_RESULT_TABLE = "test_match_op_result_db"; + + private static final String MONGODB_SPLIT_RESULT_TABLE = "test_split_op_result_db"; + + private static final String MONGODB_SINK_TABLE = "test_source_sink_table"; + + private static final String MONGODB_UPDATE_TABLE = "test_update_table"; + + private static final String MONGODB_FLAT_TABLE = "test_flat_table"; private GenericContainer mongodbContainer; + private MongoClient client; @TestTemplate - public void testMongodbSourceToAssertSink(TestContainer container) + public void testMongodbSourceAndSink(TestContainer container) throws IOException, InterruptedException { - Container.ExecResult execResult = container.executeJob("/mongodb_source_to_assert.conf"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + Container.ExecResult insertResult = container.executeJob("/fake_source_to_mongodb.conf"); + Assertions.assertEquals(0, insertResult.getExitCode(), insertResult.getStderr()); + + Container.ExecResult assertResult = container.executeJob("/mongodb_source_to_assert.conf"); + Assertions.assertEquals(0, assertResult.getExitCode(), assertResult.getStderr()); + clearDate(MONGODB_SINK_TABLE); } @TestTemplate - public void testMongodb(TestContainer container) throws IOException, InterruptedException { - Container.ExecResult execResult = container.executeJob("/mongodb_source_and_sink.conf"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + public void testMongodbSourceMatch(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult queryResult = + container.executeJob("/matchIT/mongodb_matchQuery_source_to_assert.conf"); + Assertions.assertEquals(0, queryResult.getExitCode(), queryResult.getStderr()); + + Assertions.assertIterableEquals( + TEST_MATCH_DATASET.stream() + .filter(x -> x.get("c_int").equals(2)) + .peek(e -> e.remove("_id")) + .collect(Collectors.toList()), + readMongodbData(MONGODB_MATCH_RESULT_TABLE).stream() + .peek(e -> e.remove("_id")) + .collect(Collectors.toList())); + clearDate(MONGODB_MATCH_RESULT_TABLE); + + Container.ExecResult projectionResult = + container.executeJob("/matchIT/mongodb_matchProjection_source_to_assert.conf"); + Assertions.assertEquals(0, projectionResult.getExitCode(), projectionResult.getStderr()); + + Assertions.assertIterableEquals( + TEST_MATCH_DATASET.stream() + .map(Document::new) + .peek(document -> document.remove("c_bigint")) + .peek(e -> e.remove("_id")) + .collect(Collectors.toList()), + readMongodbData(MONGODB_MATCH_RESULT_TABLE).stream() + .peek(e -> e.remove("_id")) + .collect(Collectors.toList())); + clearDate(MONGODB_MATCH_RESULT_TABLE); } @TestTemplate - public void testMongodbMatchQuery(TestContainer container) + public void testFakeSourceToUpdateMongodb(TestContainer container) throws IOException, InterruptedException { - Container.ExecResult execResult = - container.executeJob("/mongodb_source_matchQuery_and_sink.conf"); - Assertions.assertEquals(0, execResult.getExitCode(), execResult.getStderr()); + + Container.ExecResult insertResult = + container.executeJob("/updateIT/fake_source_to_updateMode_insert_mongodb.conf"); + Assertions.assertEquals(0, insertResult.getExitCode(), insertResult.getStderr()); + + Container.ExecResult updateResult = + container.executeJob("/updateIT/fake_source_to_update_mongodb.conf"); + Assertions.assertEquals(0, updateResult.getExitCode(), updateResult.getStderr()); + + Container.ExecResult assertResult = + container.executeJob("/updateIT/update_mongodb_to_assert.conf"); + Assertions.assertEquals(0, assertResult.getExitCode(), assertResult.getStderr()); + + clearDate(MONGODB_UPDATE_TABLE); + } + + @TestTemplate + public void testFlatSyncString(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult insertResult = + container.executeJob("/flatIT/fake_source_to_flat_mongodb.conf"); + Assertions.assertEquals(0, insertResult.getExitCode(), insertResult.getStderr()); + + Container.ExecResult assertResult = + container.executeJob("/flatIT/mongodb_flat_source_to_assert.conf"); + Assertions.assertEquals(0, assertResult.getExitCode(), assertResult.getStderr()); + + clearDate(MONGODB_FLAT_TABLE); + } + + @TestTemplate + public void testMongodbSourceSplit(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult queryResult = + container.executeJob("/splitIT/mongodb_split_key_source_to_assert.conf"); + Assertions.assertEquals(0, queryResult.getExitCode(), queryResult.getStderr()); + + Assertions.assertIterableEquals( + TEST_SPLIT_DATASET.stream() + .map(Document::new) + .peek(e -> e.remove("_id")) + .collect(Collectors.toList()), + readMongodbData(MONGODB_SPLIT_RESULT_TABLE).stream() + .peek(e -> e.remove("_id")) + .collect(Collectors.toList())); + clearDate(MONGODB_SPLIT_RESULT_TABLE); + + Container.ExecResult projectionResult = + container.executeJob("/splitIT/mongodb_split_size_source_to_assert.conf"); + Assertions.assertEquals(0, projectionResult.getExitCode(), projectionResult.getStderr()); + + Assertions.assertIterableEquals( + TEST_SPLIT_DATASET.stream() + .map(Document::new) + .peek(e -> e.remove("_id")) + .collect(Collectors.toList()), + readMongodbData(MONGODB_SPLIT_RESULT_TABLE).stream() + .peek(e -> e.remove("_id")) + .collect(Collectors.toList())); + clearDate(MONGODB_SPLIT_RESULT_TABLE); } public void initConnection() { @@ -106,73 +206,24 @@ public void initConnection() { client = MongoClients.create(url); } - private void initSourceData(String database, String table, List documents) { - MongoCollection sourceTable = client.getDatabase(database).getCollection(table); + private void initSourceData() { + MongoCollection sourceMatchTable = + client.getDatabase(MongodbIT.MONGODB_DATABASE) + .getCollection(MongodbIT.MONGODB_MATCH_TABLE); - sourceTable.deleteMany(new Document()); - sourceTable.insertMany(documents); - } + sourceMatchTable.deleteMany(new Document()); + sourceMatchTable.insertMany(MongodbIT.TEST_MATCH_DATASET); - private static List generateTestDataSet(int start, int end) { - SeaTunnelRowType seatunnelRowType = - new SeaTunnelRowType( - new String[] { - "id", - "c_map", - "c_array", - "c_string", - "c_boolean", - "c_tinyint", - "c_smallint", - "c_int", - "c_bigint", - "c_float", - "c_double", - "c_decimal", - "c_bytes", - "c_date" - }, - new SeaTunnelDataType[] { - BasicType.LONG_TYPE, - new MapType(BasicType.STRING_TYPE, BasicType.SHORT_TYPE), - ArrayType.BYTE_ARRAY_TYPE, - BasicType.STRING_TYPE, - BasicType.BOOLEAN_TYPE, - BasicType.BYTE_TYPE, - BasicType.SHORT_TYPE, - BasicType.INT_TYPE, - BasicType.LONG_TYPE, - BasicType.FLOAT_TYPE, - BasicType.DOUBLE_TYPE, - new DecimalType(2, 1), - PrimitiveByteArrayType.INSTANCE, - LocalTimeType.LOCAL_DATE_TYPE - }); - Serializer serializer = new DefaultSerializer(seatunnelRowType); + MongoCollection sourceSplitTable = + client.getDatabase(MongodbIT.MONGODB_DATABASE) + .getCollection(MongodbIT.MONGODB_SPLIT_TABLE); - List documents = new ArrayList<>(); - for (int i = start; i < end; i++) { - SeaTunnelRow row = - new SeaTunnelRow( - new Object[] { - Long.valueOf(i), - Collections.singletonMap("key", Short.parseShort("1")), - new Byte[] {Byte.parseByte("1")}, - "string", - Boolean.FALSE, - Byte.parseByte("1"), - Short.parseShort("1"), - Integer.parseInt("1"), - Long.parseLong("1"), - Float.parseFloat("1.1"), - Double.parseDouble("1.1"), - BigDecimal.valueOf(11, 1), - "test".getBytes(), - LocalDate.now() - }); - documents.add(serializer.serialize(row)); - } - return documents; + sourceSplitTable.deleteMany(new Document()); + sourceSplitTable.insertMany(MongodbIT.TEST_SPLIT_DATASET); + } + + private void clearDate(String table) { + client.getDatabase(MONGODB_DATABASE).getCollection(table).drop(); } @BeforeAll @@ -194,6 +245,7 @@ public void startUp() { .withStartupTimeout(Duration.ofMinutes(2))) .withLogConsumer( new Slf4jLogConsumer(DockerLoggerFactory.getLogger(MONGODB_IMAGE))); + mongodbContainer.setPortBindings(Collections.singletonList("27017:27017")); Startables.deepStart(Stream.of(mongodbContainer)).join(); log.info("Mongodb container started"); @@ -203,10 +255,83 @@ public void startUp() { .pollInterval(500, TimeUnit.MILLISECONDS) .atMost(180, TimeUnit.SECONDS) .untilAsserted(this::initConnection); - this.initSourceData(MONGODB_DATABASE, MONGODB_SOURCE_TABLE, TEST_DATASET); + this.initSourceData(); + } + + public static List generateTestDataSet(int count) { + List dataSet = new ArrayList<>(); + + for (int i = 0; i < count; i++) { + dataSet.add( + new Document( + "c_map", + new Document("OQBqH", randomString()) + .append("rkvlO", randomString()) + .append("pCMEX", randomString()) + .append("DAgdj", randomString()) + .append("dsJag", randomString())) + .append( + "c_array", + Arrays.asList( + RANDOM.nextInt(), + RANDOM.nextInt(), + RANDOM.nextInt(), + RANDOM.nextInt(), + RANDOM.nextInt())) + .append("c_string", randomString()) + .append("c_boolean", RANDOM.nextBoolean()) + .append("c_int", i) + .append("c_bigint", RANDOM.nextLong()) + .append("c_double", RANDOM.nextDouble() * Double.MAX_VALUE) + .append( + "c_row", + new Document( + "c_map", + new Document("OQBqH", randomString()) + .append("rkvlO", randomString()) + .append("pCMEX", randomString()) + .append("DAgdj", randomString()) + .append("dsJag", randomString())) + .append( + "c_array", + Arrays.asList( + RANDOM.nextInt(), + RANDOM.nextInt(), + RANDOM.nextInt(), + RANDOM.nextInt(), + RANDOM.nextInt())) + .append("c_string", randomString()) + .append("c_boolean", RANDOM.nextBoolean()) + .append("c_int", RANDOM.nextInt()) + .append("c_bigint", RANDOM.nextLong()) + .append( + "c_double", + RANDOM.nextDouble() * Double.MAX_VALUE))); + } + return dataSet; + } + + private static String randomString() { + int length = RANDOM.nextInt(10) + 1; + StringBuilder sb = new StringBuilder(length); + for (int i = 0; i < length; i++) { + char c = (char) (RANDOM.nextInt(26) + 'a'); + sb.append(c); + } + return sb.toString(); + } + + private List readMongodbData(String collection) { + MongoCollection sinkTable = + client.getDatabase(MONGODB_DATABASE).getCollection(collection); + MongoCursor cursor = sinkTable.find().sort(Sorts.ascending("c_int")).cursor(); + List documents = new ArrayList<>(); + while (cursor.hasNext()) { + documents.add(cursor.next()); + } + return documents; } - @AfterAll @Override public void tearDown() { if (client != null) { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/fake_source_to_mongodb.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/fake_source_to_mongodb.conf new file mode 100644 index 00000000000..3ea58f960c3 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/fake_source_to_mongodb.conf @@ -0,0 +1,100 @@ +# +# 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" + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FakeSource { + row.num = 5 + int.template = [2] + result_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true" + database = "test_db" + collection = "test_source_sink_table" + transaction-enable = true + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } + } + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/flatIT/fake_source_to_flat_mongodb.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/flatIT/fake_source_to_flat_mongodb.conf new file mode 100644 index 00000000000..fb37fc99293 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/flatIT/fake_source_to_flat_mongodb.conf @@ -0,0 +1,99 @@ +# +# 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" + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FakeSource { + result_table_name = "mongodb_table" + row.num = 1 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true" + database = "test_db" + collection = "test_flat_table" + source_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } + } + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_and_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/flatIT/mongodb_flat_source_to_assert.conf similarity index 56% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_and_sink.conf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/flatIT/mongodb_flat_source_to_assert.conf index d3c60494cb6..54ad13d75ff 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_and_sink.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/flatIT/mongodb_flat_source_to_assert.conf @@ -15,16 +15,10 @@ # limitations under the License. # -###### -###### This config file is a demonstration of batch processing in seatunnel config -###### - env { - # You can set flink configuration here execution.parallelism = 1 job.mode = "BATCH" - - # You can set spark configuration here + #spark config spark.app.name = "SeaTunnel" spark.executor.instances = 1 spark.executor.cores = 1 @@ -34,37 +28,47 @@ env { source { MongoDB { - uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true&writeConcern=majority" + uri = "mongodb://e2e_mongodb:27017/test_db" database = "test_db" - collection = "source_table" + collection = "test_flat_table" + result_table_name = "mongodb_table" + flat.sync-string = true schema = { fields { - id = bigint - 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(2, 1)" - c_bytes = bytes - c_date = date + data = string } } } } -transform { -} - sink { - MongoDB { - uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true&writeConcern=majority" - database = "test_db" - collection = "sink_table" + Console { + source_table_name = "mongodb_table" + } + Assert { + source_table_name = "mongodb_table" + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 1 + }, + { + rule_type = MIN_ROW + rule_value = 1 + } + ], + field_rules = [ + { + field_name = data + field_type = string + field_value = [ + { + rule_type = NOT_NULL + } + ] + } + ] + } } } \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/matchIT/mongodb_matchProjection_source_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/matchIT/mongodb_matchProjection_source_to_assert.conf new file mode 100644 index 00000000000..da73fafd16d --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/matchIT/mongodb_matchProjection_source_to_assert.conf @@ -0,0 +1,91 @@ +# +# 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" + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db" + database = "test_db" + collection = "test_match_op_db" + match.projection = "{ c_bigint:0 }" + result_table_name = "mongodb_table" + cursor.no-timeout = true + fetch.size = 1000 + max.time-min = 100 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_double = double + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + } + } + } + } +} + +sink { + Console { + source_table_name = "mongodb_table" + } + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true" + database = "test_db" + collection = "test_match_op_result_db" + source_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_double = double + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + } + } + } + } + +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_matchQuery_and_sink.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/matchIT/mongodb_matchQuery_source_to_assert.conf similarity index 50% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_matchQuery_and_sink.conf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/matchIT/mongodb_matchQuery_source_to_assert.conf index 06657ac62e1..197c58ee80f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_matchQuery_and_sink.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/matchIT/mongodb_matchQuery_source_to_assert.conf @@ -15,16 +15,10 @@ # limitations under the License. # -###### -###### This config file is a demonstration of batch processing in seatunnel config -###### - env { - # You can set flink configuration here execution.parallelism = 1 job.mode = "BATCH" - - # You can set spark configuration here + #spark config spark.app.name = "SeaTunnel" spark.executor.instances = 1 spark.executor.cores = 1 @@ -34,38 +28,65 @@ env { source { MongoDB { - uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true&writeConcern=majority" + uri = "mongodb://e2e_mongodb:27017/test_db" database = "test_db" - collection = "source_matchQuery_table" - matchQuery = "{"id":3}" + collection = "test_match_op_db" + result_table_name = "mongodb_table" + match.query = "{c_int: 2}" + cursor.no-timeout = true + fetch.size = 1000 + max.time-min = 100 schema = { fields { - id = bigint - c_map = "map" - c_array = "array" + 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(2, 1)" - c_bytes = bytes - c_date = date + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + } } } } } -transform { -} - sink { + Console { + source_table_name = "mongodb_table" + } MongoDB { - uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true&writeConcern=majority" + uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true" database = "test_db" - collection = "sink_matchQuery_table" + collection = "test_match_op_result_db" + source_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + } + } + } } -} \ No newline at end of file +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_to_assert.conf index ad5c1a771ff..4c59dee7b52 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_to_assert.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/mongodb_source_to_assert.conf @@ -14,60 +14,102 @@ # 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 flink configuration here execution.parallelism = 1 job.mode = "BATCH" - #execution.checkpoint.interval = 10000 - #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local } source { MongoDB { - uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true&writeConcern=majority" + uri = "mongodb://e2e_mongodb:27017/test_db" database = "test_db" - collection = "source_table" + collection = "test_source_sink_table" + cursor.no-timeout = true + result_table_name = "mongodb_table" schema = { fields { - id = bigint - c_map = "map" - c_array = "array" + 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(2, 1)" c_bytes = bytes c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } } } } } sink { + Console { + source_table_name = "mongodb_table" + } Assert { - rules = - { - row_rules = [ - { - rule_type = MAX_ROW - rule_value = 10 - }, - { - rule_type = MIN_ROW - rule_value = 10 - } - ] - } + source_table_name = "mongodb_table" + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + }, + { + rule_type = MIN_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 + } + ] + } + ] + } } - # 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/connector-v2/sink/Assert } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/splitIT/mongodb_split_key_source_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/splitIT/mongodb_split_key_source_to_assert.conf new file mode 100644 index 00000000000..6dcc339c54c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/splitIT/mongodb_split_key_source_to_assert.conf @@ -0,0 +1,92 @@ +# +# 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" + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db" + database = "test_db" + collection = "test_split_op_db" + result_table_name = "mongodb_table" + partition.split-key = "c_int" + cursor.no-timeout = true + fetch.size = 1000 + max.time-min = 100 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + } + } + } + } +} + +sink { + Console { + source_table_name = "mongodb_table" + } + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true" + database = "test_db" + collection = "test_split_op_result_db" + source_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + } + } + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/splitIT/mongodb_split_size_source_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/splitIT/mongodb_split_size_source_to_assert.conf new file mode 100644 index 00000000000..e1fc6454696 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/splitIT/mongodb_split_size_source_to_assert.conf @@ -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. +# + +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 { + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db" + database = "test_db" + collection = "test_split_op_db" + result_table_name = "mongodb_table" + partition.split-key = c_int + partition.split-size = 1024 + cursor.no-timeout = true + fetch.size = 1000 + max.time-min = 100 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + } + } + } + } +} + +sink { + Console { + source_table_name = "mongodb_table" + } + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true" + database = "test_db" + collection = "test_split_op_result_db" + source_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + } + } + } + } + +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/updateIT/fake_source_to_updateMode_insert_mongodb.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/updateIT/fake_source_to_updateMode_insert_mongodb.conf new file mode 100644 index 00000000000..0ecc9dfb364 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/updateIT/fake_source_to_updateMode_insert_mongodb.conf @@ -0,0 +1,102 @@ +# +# 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" + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FakeSource { + row.num = 5 + int.template = [2] + result_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true" + database = "test_db" + collection = "test_update_table" + upsert-enable = true + upsert-key = ["c_string"] + source_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } + } + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/updateIT/fake_source_to_update_mongodb.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/updateIT/fake_source_to_update_mongodb.conf new file mode 100644 index 00000000000..c210e9eabf9 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/updateIT/fake_source_to_update_mongodb.conf @@ -0,0 +1,102 @@ +# +# 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" + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + FakeSource { + row.num = 5 + int.template = [2] + result_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db?retryWrites=true" + database = "test_db" + collection = "test_update_table" + upsert-enable = true + upsert-key = ["c_int"] + source_table_name = "mongodb_table" + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } + } + } + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/updateIT/update_mongodb_to_assert.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/updateIT/update_mongodb_to_assert.conf new file mode 100644 index 00000000000..64a7db34078 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/updateIT/update_mongodb_to_assert.conf @@ -0,0 +1,117 @@ +# +# 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" + #spark config + spark.app.name = "SeaTunnel" + spark.executor.instances = 1 + spark.executor.cores = 1 + spark.executor.memory = "1g" + spark.master = local +} + +source { + MongoDB { + uri = "mongodb://e2e_mongodb:27017/test_db" + database = "test_db" + collection = "test_update_table" + result_table_name = "mongodb_table" + cursor.no-timeout = true + fetch.size = 1000 + max.time-min = 100 + schema = { + fields { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + c_row = { + c_map = "map" + c_array = "array" + c_string = string + c_boolean = boolean + c_int = int + c_bigint = bigint + c_double = double + c_bytes = bytes + c_date = date + c_decimal = "decimal(33, 18)" + c_timestamp = timestamp + } + } + } + } +} + +sink { + Console { + source_table_name = "mongodb_table" + } + Assert { + source_table_name = "mongodb_table" + rules { + row_rules = [ + { + rule_type = MAX_ROW + rule_value = 5 + }, + { + rule_type = MIN_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 + } + ] + } + ] + } + } +} +