diff --git a/config/plugin_config b/config/plugin_config index 98c9fa8c2c69..95b952b31bf1 100644 --- a/config/plugin_config +++ b/config/plugin_config @@ -24,6 +24,7 @@ connector-amazondynamodb connector-assert connector-cassandra connector-cdc-mysql +connector-cdc-mongodb connector-cdc-sqlserver connector-clickhouse connector-datahub diff --git a/docs/en/connector-v2/source/MongoDB-CDC.md b/docs/en/connector-v2/source/MongoDB-CDC.md new file mode 100644 index 000000000000..1422396be998 --- /dev/null +++ b/docs/en/connector-v2/source/MongoDB-CDC.md @@ -0,0 +1,183 @@ +# MongoDB CDC + +> MongoDB CDC source connector + +Support Those Engines +--------------------- + +> SeaTunnel Zeta
+ +Key Features +------------ + +- [ ] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) + +Description +----------- + +The MongoDB CDC connector allows for reading snapshot data and incremental data from MongoDB database. + +Supported DataSource Info +------------------------- + +In order to use the Mongodb connector, the following dependencies are required. +They can be downloaded via install-plugin.sh or from the Maven central repository. + +| Datasource | Supported Versions | Dependency | +|------------|--------------------|-------------------------------------------------------------------------------------------------------------------| +| MongoDB | universal | [Download](https://mvnrepository.com/artifact/org.apache.seatunnel/seatunnel-connectors-v2/connector-cdc-mongodb) | + +Availability Settings +--------------------- + +1.MongoDB version: MongoDB version >= 4.0. + +2.Cluster deployment: replica sets or sharded clusters. + +3.Storage Engine: WiredTiger Storage Engine. + +4.Permissions:changeStream and read + +```shell +use admin; +db.createRole( + { + role: "strole", + privileges: [{ + resource: { db: "", collection: "" }, + actions: [ + "splitVector", + "listDatabases", + "listCollections", + "collStats", + "find", + "changeStream" ] + }], + roles: [ + { role: 'read', db: 'config' } + ] + } +); + +db.createUser( + { + user: 'stuser', + pwd: 'stpw', + roles: [ + { role: 'strole', db: 'admin' } + ] + } +); +``` + +Data Type Mapping +----------------- + +The following table lists the field data type mapping from MongoDB BSON type to Seatunnel data type. + +| MongoDB BSON type | Seatunnel Data type | +|-------------------|---------------------| +| ObjectId | STRING | +| String | STRING | +| Boolean | BOOLEAN | +| Binary | BINARY | +| Int32 | INTEGER | +| Int64 | BIGINT | +| Double | DOUBLE | +| Decimal128 | DECIMAL | +| Date | Date | +| Timestamp | Timestamp | +| Object | ROW | +| Array | ARRAY | + +For specific types in MongoDB, we use Extended JSON format to map them to Seatunnel STRING type. + +| MongoDB BSON type | Seatunnel STRING | +|-------------------|----------------------------------------------------------------------------------------------| +| Symbol | {"_value": {"$symbol": "12"}} | +| RegularExpression | {"_value": {"$regularExpression": {"pattern": "^9$", "options": "i"}}} | +| JavaScript | {"_value": {"$code": "function() { return 10; }"}} | +| DbPointer | {"_value": {"$dbPointer": {"$ref": "db.coll", "$id": {"$oid": "63932a00da01604af329e33c"}}}} | + +**Tips** + +> 1.When using the DECIMAL type in SeaTunnel, be aware that the maximum range cannot exceed 34 digits, which means you should use decimal(34, 18).
+ +Source Options +-------------- + +| Name | Type | Required | Default | Description | +|------------------------------------|--------|----------|---------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| hosts | String | Yes | - | The comma-separated list of hostname and port pairs of the MongoDB servers. eg. `localhost:27017,localhost:27018` | +| username | String | No | - | Name of the database user to be used when connecting to MongoDB. | +| password | String | No | - | Password to be used when connecting to MongoDB. | +| databases | String | Yes | - | Name of the database to watch for changes. If not set then all databases will be captured. The database also supports regular expressions to monitor multiple databases matching the regular expression. eg. `db1,db2` | +| collections | String | Yes | - | Name of the collection in the database to watch for changes. If not set then all collections will be captured. The collection also supports regular expressions to monitor multiple collections matching fully-qualified collection identifiers. eg. `db1.coll1,db2.coll2` | +| connection.options | String | No | - | The ampersand-separated connection options of MongoDB. eg. `replicaSet=test&connectTimeoutMS=300000` | +| batch.size | Long | No | 1024 | The cursor batch size. | +| poll.max.batch.size | Enum | No | 1024 | Maximum number of change stream documents to include in a single batch when polling for new data. | +| poll.await.time.ms | Long | No | 1000 | The amount of time to wait before checking for new results on the change stream. | +| heartbeat.interval.ms | String | No | 0 | The length of time in milliseconds between sending heartbeat messages. Use 0 to disable. | +| incremental.snapshot.chunk.size.mb | Long | No | 64 | The chunk size mb of incremental snapshot. | + +**Tips:** + +> 1.If the collection changes at a slow pace, it is strongly recommended to set an appropriate value greater than 0 for the heartbeat.interval.ms parameter. When we recover a Seatunnel job from a checkpoint or savepoint, the heartbeat events can push the resumeToken forward to avoid its expiration.
+> 2.MongoDB has a limit of 16MB for a single document. Change documents include additional information, so even if the original document is not larger than 15MB, the change document may exceed the 16MB limit, resulting in the termination of the Change Stream operation.
+> 3.It is recommended to use immutable shard keys. In MongoDB, shard keys allow modifications after transactions are enabled, but changing the shard key can cause frequent shard migrations, resulting in additional performance overhead. Additionally, modifying the shard key can also cause the Update Lookup feature to become ineffective, leading to inconsistent results in CDC (Change Data Capture) scenarios.
+ +#### example + +```conf +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + execution.checkpoint.interval = 5000 +} + +source { + MongoDB-CDC { + hosts = "mongo0:27017" + database = "inventory" + collection = "inventory.products" + username = stuser + password = stpw + schema = { + fields { + "_id" : string, + "name" : string, + "description" : string, + "weight" : string + } + } + } +} + +sink { + jdbc { + url = "jdbc:mysql://mysql_cdc_e2e:3306" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user" + password = "seatunnel" + + generate_sink_sql = true + # You need to configure both database and table + database = mongodb_cdc + table = products + primary_keys = ["_id"] + } +} +``` + +## Changelog + +- Add MongoDB CDC Source Connector + +### next version + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index de6593b4523c..3943159aefaf 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -99,6 +99,7 @@ seatunnel.sink.Doris = connector-doris seatunnel.source.Maxcompute = connector-maxcompute seatunnel.sink.Maxcompute = connector-maxcompute seatunnel.source.MySQL-CDC = connector-cdc-mysql +seatunnel.source.MongoDB-CDC = connector-cdc-mongodb seatunnel.sink.S3Redshift = connector-s3-redshift seatunnel.source.TDengine = connector-tdengine seatunnel.sink.TDengine = connector-tdengine diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/pom.xml b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/pom.xml new file mode 100644 index 000000000000..77d0301f0efa --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/pom.xml @@ -0,0 +1,95 @@ + + + + 4.0.0 + + org.apache.seatunnel + connector-cdc + ${revision} + + connector-cdc-mongodb + SeaTunnel : Connectors V2 : CDC : Mongodb + + + 4.7.1 + 1.11.1 + 1.10.1 + 1.6.4.Final + 24.0.1 + 4.13.2 + + + + + org.apache.seatunnel + connector-cdc-base + ${project.version} + compile + + + io.debezium + debezium-connector-mongodb + ${debezium.version} + compile + + + org.mongodb.kafka + mongo-kafka-connect + ${mongo-kafka-connect.version} + + + org.mongodb + mongodb-driver-sync + + + org.apache.kafka + connect-api + + + org.apache.avro + avro + + + + + org.apache.avro + avro + ${avro.version} + + + org.mongodb + mongodb-driver-sync + ${mongo.driver.version} + + + junit + junit + ${junit.vserion} + test + + + org.jetbrains + annotations + ${annotations.vserion} + compile + + + diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSource.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSource.java new file mode 100644 index 000000000000..30cb68865bd0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSource.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SupportParallelism; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; +import org.apache.seatunnel.connectors.cdc.base.dialect.DataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.option.StopMode; +import org.apache.seatunnel.connectors.cdc.base.source.IncrementalSource; +import org.apache.seatunnel.connectors.cdc.base.source.offset.OffsetFactory; +import org.apache.seatunnel.connectors.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.seatunnel.connectors.cdc.debezium.row.DebeziumJsonDeserializeSchema; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfigProvider; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.sender.MongoDBConnectorDeserializationSchema; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.dialect.MongodbDialect; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffsetFactory; + +import org.jetbrains.annotations.NotNull; + +import com.google.auto.service.AutoService; +import lombok.NoArgsConstructor; + +import java.util.Optional; + +@NoArgsConstructor +@AutoService(SeaTunnelSource.class) +public class MongodbIncrementalSource extends IncrementalSource + implements SupportParallelism { + + static final String IDENTIFIER = "MongoDB-CDC"; + + public MongodbIncrementalSource( + ReadonlyConfig options, SeaTunnelDataType dataType) { + super(options, dataType); + } + + @Override + public Option getStartupModeOption() { + return MongodbSourceOptions.STARTUP_MODE; + } + + @Override + public Option getStopModeOption() { + return MongodbSourceOptions.STOP_MODE; + } + + @Override + public String getPluginName() { + return IDENTIFIER; + } + + @Override + public SourceConfig.Factory createSourceConfigFactory( + @NotNull ReadonlyConfig config) { + MongodbSourceConfigProvider.Builder builder = + MongodbSourceConfigProvider.newBuilder() + .hosts(config.get(MongodbSourceOptions.HOSTS)) + .validate(); + Optional.ofNullable(config.get(MongodbSourceOptions.DATABASE)) + .ifPresent(builder::databaseList); + Optional.ofNullable(config.get(MongodbSourceOptions.COLLECTION)) + .ifPresent(builder::collectionList); + Optional.ofNullable(config.get(MongodbSourceOptions.USERNAME)).ifPresent(builder::username); + Optional.ofNullable(config.get(MongodbSourceOptions.PASSWORD)).ifPresent(builder::password); + Optional.ofNullable(config.get(MongodbSourceOptions.CONNECTION_OPTIONS)) + .ifPresent(builder::connectionOptions); + Optional.ofNullable(config.get(MongodbSourceOptions.BATCH_SIZE)) + .ifPresent(builder::batchSize); + Optional.ofNullable(config.get(MongodbSourceOptions.POLL_MAX_BATCH_SIZE)) + .ifPresent(builder::pollMaxBatchSize); + Optional.ofNullable(config.get(MongodbSourceOptions.POLL_AWAIT_TIME_MILLIS)) + .ifPresent(builder::pollAwaitTimeMillis); + Optional.ofNullable(config.get(MongodbSourceOptions.HEARTBEAT_INTERVAL_MILLIS)) + .ifPresent(builder::heartbeatIntervalMillis); + Optional.ofNullable(config.get(MongodbSourceOptions.HEARTBEAT_INTERVAL_MILLIS)) + .ifPresent(builder::splitMetaGroupSize); + Optional.ofNullable(config.get(MongodbSourceOptions.INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB)) + .ifPresent(builder::splitSizeMB); + Optional.ofNullable(startupConfig).ifPresent(builder::startupOptions); + Optional.ofNullable(stopConfig).ifPresent(builder::stopOptions); + return builder; + } + + @SuppressWarnings("unchecked") + @Override + public DebeziumDeserializationSchema createDebeziumDeserializationSchema( + ReadonlyConfig config) { + SeaTunnelDataType physicalRowType; + if (dataType == null) { + return (DebeziumDeserializationSchema) + new DebeziumJsonDeserializeSchema( + config.get(MongodbSourceOptions.DEBEZIUM_PROPERTIES)); + } else { + physicalRowType = dataType; + return (DebeziumDeserializationSchema) + new MongoDBConnectorDeserializationSchema(physicalRowType, physicalRowType); + } + } + + @Override + public DataSourceDialect createDataSourceDialect(ReadonlyConfig config) { + return new MongodbDialect(); + } + + @Override + public OffsetFactory createOffsetFactory(ReadonlyConfig config) { + return new ChangeStreamOffsetFactory(); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSourceFactory.java new file mode 100644 index 000000000000..b0e164832109 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/MongodbIncrementalSourceFactory.java @@ -0,0 +1,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. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.SupportMultipleTable; +import org.apache.seatunnel.api.table.factory.TableFactoryContext; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.type.MultipleRowType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions; + +import org.jetbrains.annotations.NotNull; + +import com.google.auto.service.AutoService; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +@AutoService(Factory.class) +public class MongodbIncrementalSourceFactory implements TableSourceFactory, SupportMultipleTable { + @Override + public String factoryIdentifier() { + return MongodbIncrementalSource.IDENTIFIER; + } + + @Override + public OptionRule optionRule() { + return MongodbSourceOptions.getBaseRule() + .required( + MongodbSourceOptions.HOSTS, + MongodbSourceOptions.DATABASE, + MongodbSourceOptions.COLLECTION) + .optional( + MongodbSourceOptions.USERNAME, + MongodbSourceOptions.PASSWORD, + MongodbSourceOptions.CONNECTION_OPTIONS, + MongodbSourceOptions.BATCH_SIZE, + MongodbSourceOptions.POLL_MAX_BATCH_SIZE, + MongodbSourceOptions.POLL_AWAIT_TIME_MILLIS, + MongodbSourceOptions.HEARTBEAT_INTERVAL_MILLIS, + MongodbSourceOptions.INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB) + .build(); + } + + @Override + public Class getSourceClass() { + return MongodbIncrementalSource.class; + } + + @SuppressWarnings("unchecked") + @Override + public + TableSource createSource(TableFactoryContext context) { + return () -> { + SeaTunnelDataType dataType; + if (context.getCatalogTables().size() == 1) { + dataType = + context.getCatalogTables().get(0).getTableSchema().toPhysicalRowDataType(); + } else { + Map rowTypeMap = new HashMap<>(); + for (CatalogTable catalogTable : context.getCatalogTables()) { + rowTypeMap.put( + catalogTable.getTableId().toTablePath().toString(), + catalogTable.getTableSchema().toPhysicalRowDataType()); + } + dataType = new MultipleRowType(rowTypeMap); + } + return (SeaTunnelSource) + new MongodbIncrementalSource<>(context.getOptions(), dataType); + }; + } + + @Override + public Result applyTables(@NotNull TableFactoryContext context) { + return Result.of(context.getCatalogTables(), Collections.emptyList()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfig.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfig.java new file mode 100644 index 000000000000..594296c6d057 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfig.java @@ -0,0 +1,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. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config; + +import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StartupConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StopConfig; + +import lombok.EqualsAndHashCode; +import lombok.Getter; + +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.buildConnectionString; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; + +@Getter +@EqualsAndHashCode +public class MongodbSourceConfig implements SourceConfig { + + private static final long serialVersionUID = 1L; + + private final String hosts; + + private final String username; + + private final String password; + + private final List databaseList; + + private final List collectionList; + + private final String connectionString; + + private final int batchSize; + + private final int pollAwaitTimeMillis; + + private final int pollMaxBatchSize; + + private final boolean updateLookup; + + private final StartupConfig startupOptions; + + private final StopConfig stopOptions; + + private final int heartbeatIntervalMillis; + + private final int splitMetaGroupSize; + + private final int splitSizeMB; + + MongodbSourceConfig( + String hosts, + String username, + String password, + List databaseList, + List collectionList, + String connectionOptions, + int batchSize, + int pollAwaitTimeMillis, + int pollMaxBatchSize, + boolean updateLookup, + StartupConfig startupOptions, + StopConfig stopOptions, + int heartbeatIntervalMillis, + int splitMetaGroupSize, + int splitSizeMB) { + this.hosts = checkNotNull(hosts); + this.username = username; + this.password = password; + this.databaseList = databaseList; + this.collectionList = collectionList; + this.connectionString = + buildConnectionString(username, password, hosts, connectionOptions) + .getConnectionString(); + this.batchSize = batchSize; + this.pollAwaitTimeMillis = pollAwaitTimeMillis; + this.pollMaxBatchSize = pollMaxBatchSize; + this.updateLookup = updateLookup; + this.startupOptions = startupOptions; + this.stopOptions = stopOptions; + this.heartbeatIntervalMillis = heartbeatIntervalMillis; + this.splitMetaGroupSize = splitMetaGroupSize; + this.splitSizeMB = splitSizeMB; + } + + @Override + public StartupConfig getStartupConfig() { + return startupOptions; + } + + @Override + public StopConfig getStopConfig() { + return stopOptions; + } + + @Override + public int getSplitSize() { + return splitSizeMB; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfigProvider.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfigProvider.java new file mode 100644 index 000000000000..194e4de6f011 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceConfigProvider.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config; + +import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StartupConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StopConfig; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.option.StopMode; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; + +import java.util.List; +import java.util.Objects; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.BATCH_SIZE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.HEARTBEAT_INTERVAL_MILLIS; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.POLL_AWAIT_TIME_MILLIS; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.POLL_MAX_BATCH_SIZE; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkArgument; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; + +public class MongodbSourceConfigProvider { + + private MongodbSourceConfigProvider() {} + + @NotNull @Contract(" -> new") + public static Builder newBuilder() { + return new Builder(); + } + + public static class Builder implements SourceConfig.Factory { + private String hosts; + private String username; + private String password; + private List databaseList; + private List collectionList; + private String connectionOptions; + private int batchSize = BATCH_SIZE.defaultValue(); + private int pollAwaitTimeMillis = POLL_AWAIT_TIME_MILLIS.defaultValue(); + private int pollMaxBatchSize = POLL_MAX_BATCH_SIZE.defaultValue(); + private StartupConfig startupOptions; + private StopConfig stopOptions; + private int heartbeatIntervalMillis = HEARTBEAT_INTERVAL_MILLIS.defaultValue(); + private int splitMetaGroupSize = 2; + private int splitSizeMB = INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB.defaultValue(); + + public Builder hosts(String hosts) { + this.hosts = hosts; + return this; + } + + public Builder connectionOptions(String connectionOptions) { + this.connectionOptions = connectionOptions; + return this; + } + + public Builder username(String username) { + this.username = username; + return this; + } + + public Builder password(String password) { + this.password = password; + return this; + } + + public Builder databaseList(List databases) { + this.databaseList = databases; + return this; + } + + public Builder collectionList(List collections) { + this.collectionList = collections; + return this; + } + + public Builder batchSize(int batchSize) { + checkArgument(batchSize >= 0); + this.batchSize = batchSize; + return this; + } + + public Builder pollAwaitTimeMillis(int pollAwaitTimeMillis) { + checkArgument(pollAwaitTimeMillis > 0); + this.pollAwaitTimeMillis = pollAwaitTimeMillis; + return this; + } + + public Builder pollMaxBatchSize(int pollMaxBatchSize) { + checkArgument(pollMaxBatchSize > 0); + this.pollMaxBatchSize = pollMaxBatchSize; + return this; + } + + public Builder startupOptions(StartupConfig startupOptions) { + this.startupOptions = Objects.requireNonNull(startupOptions); + if (startupOptions.getStartupMode() != StartupMode.INITIAL + && startupOptions.getStartupMode() != StartupMode.TIMESTAMP) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unsupported startup mode " + startupOptions.getStartupMode()); + } + return this; + } + + public Builder stopOptions(StopConfig stopOptions) { + this.stopOptions = Objects.requireNonNull(stopOptions); + if (stopOptions.getStopMode() != StopMode.NEVER) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + String.format("The %s mode is not supported.", stopOptions.getStopMode())); + } + return this; + } + + public Builder heartbeatIntervalMillis(int heartbeatIntervalMillis) { + checkArgument(heartbeatIntervalMillis >= 0); + this.heartbeatIntervalMillis = heartbeatIntervalMillis; + return this; + } + + public Builder splitSizeMB(int splitSizeMB) { + checkArgument(splitSizeMB > 0); + this.splitSizeMB = splitSizeMB; + return this; + } + + public Builder splitMetaGroupSize(int splitMetaGroupSize) { + this.splitMetaGroupSize = splitMetaGroupSize; + return this; + } + + public Builder validate() { + checkNotNull(hosts, "hosts must be provided"); + return this; + } + + @Override + public MongodbSourceConfig create(int subtask) { + boolean updateLookup = true; + return new MongodbSourceConfig( + hosts, + username, + password, + databaseList, + collectionList, + connectionOptions, + batchSize, + pollAwaitTimeMillis, + pollMaxBatchSize, + updateLookup, + startupOptions, + stopOptions, + heartbeatIntervalMillis, + splitMetaGroupSize, + splitSizeMB); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java new file mode 100644 index 000000000000..df73772e0718 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/config/MongodbSourceOptions.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.connectors.cdc.base.option.SourceOptions; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.option.StopMode; + +import org.bson.BsonDouble; +import org.bson.json.JsonMode; +import org.bson.json.JsonWriterSettings; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class MongodbSourceOptions extends SourceOptions { + + public static final String ENCODE_VALUE_FIELD = "_value"; + + public static final String CLUSTER_TIME_FIELD = "clusterTime"; + + public static final String TS_MS_FIELD = "ts_ms"; + + public static final String SOURCE_FIELD = "source"; + + public static final String SNAPSHOT_FIELD = "snapshot"; + + public static final String FALSE_FALSE = "false"; + + public static final String OPERATION_TYPE_INSERT = "insert"; + + public static final String SNAPSHOT_TRUE = "true"; + + public static final String ID_FIELD = "_id"; + + public static final String DOCUMENT_KEY = "documentKey"; + + public static final String NS_FIELD = "ns"; + + public static final String OPERATION_TYPE = "operationType"; + + public static final String TIMESTAMP_FIELD = "timestamp"; + + public static final String RESUME_TOKEN_FIELD = "resumeToken"; + + public static final String FULL_DOCUMENT = "fullDocument"; + + public static final String DB_FIELD = "db"; + + public static final String COLL_FIELD = "coll"; + + public static final int FAILED_TO_PARSE_ERROR = 9; + + public static final int UNAUTHORIZED_ERROR = 13; + + public static final int ILLEGAL_OPERATION_ERROR = 20; + + public static final int UNKNOWN_FIELD_ERROR = 40415; + + public static final String DROPPED_FIELD = "dropped"; + + public static final String MAX_FIELD = "max"; + + public static final String MIN_FIELD = "min"; + + public static final String ADD_NS_FIELD_NAME = "_ns_"; + + public static final String UUID_FIELD = "uuid"; + + public static final String SHARD_FIELD = "shard"; + + public static final String DIALECT_NAME = "MongoDB"; + + public static final BsonDouble COMMAND_SUCCEED_FLAG = new BsonDouble(1.0d); + + public static final JsonWriterSettings DEFAULT_JSON_WRITER_SETTINGS = + JsonWriterSettings.builder().outputMode(JsonMode.EXTENDED).build(); + + public static final String OUTPUT_SCHEMA = + "{" + + " \"name\": \"ChangeStream\"," + + " \"type\": \"record\"," + + " \"fields\": [" + + " { \"name\": \"_id\", \"type\": \"string\" }," + + " { \"name\": \"operationType\", \"type\": [\"string\", \"null\"] }," + + " { \"name\": \"fullDocument\", \"type\": [\"string\", \"null\"] }," + + " { \"name\": \"source\"," + + " \"type\": [{\"name\": \"source\", \"type\": \"record\", \"fields\": [" + + " {\"name\": \"ts_ms\", \"type\": \"long\"}," + + " {\"name\": \"snapshot\", \"type\": [\"string\", \"null\"] } ]" + + " }, \"null\" ] }," + + " { \"name\": \"ts_ms\", \"type\": [\"long\", \"null\"]}," + + " { \"name\": \"ns\"," + + " \"type\": [{\"name\": \"ns\", \"type\": \"record\", \"fields\": [" + + " {\"name\": \"db\", \"type\": \"string\"}," + + " {\"name\": \"coll\", \"type\": [\"string\", \"null\"] } ]" + + " }, \"null\" ] }," + + " { \"name\": \"to\"," + + " \"type\": [{\"name\": \"to\", \"type\": \"record\", \"fields\": [" + + " {\"name\": \"db\", \"type\": \"string\"}," + + " {\"name\": \"coll\", \"type\": [\"string\", \"null\"] } ]" + + " }, \"null\" ] }," + + " { \"name\": \"documentKey\", \"type\": [\"string\", \"null\"] }," + + " { \"name\": \"updateDescription\"," + + " \"type\": [{\"name\": \"updateDescription\", \"type\": \"record\", \"fields\": [" + + " {\"name\": \"updatedFields\", \"type\": [\"string\", \"null\"]}," + + " {\"name\": \"removedFields\"," + + " \"type\": [{\"type\": \"array\", \"items\": \"string\"}, \"null\"]" + + " }] }, \"null\"] }," + + " { \"name\": \"clusterTime\", \"type\": [\"string\", \"null\"] }," + + " { \"name\": \"txnNumber\", \"type\": [\"long\", \"null\"]}," + + " { \"name\": \"lsid\", \"type\": [{\"name\": \"lsid\", \"type\": \"record\"," + + " \"fields\": [ {\"name\": \"id\", \"type\": \"string\"}," + + " {\"name\": \"uid\", \"type\": \"string\"}] }, \"null\"] }" + + " ]" + + "}"; + + public static final Option HOSTS = + Options.key("hosts") + .stringType() + .noDefaultValue() + .withDescription( + "The comma-separated list of hostname and port pairs of the MongoDB servers. " + + "eg. localhost:27017,localhost:27018"); + + public static final Option USERNAME = + Options.key("username") + .stringType() + .noDefaultValue() + .withDescription( + "Name of the database user to be used when connecting to MongoDB. " + + "This is required only when MongoDB is configured to use authentication."); + + public static final Option PASSWORD = + Options.key("password") + .stringType() + .noDefaultValue() + .withDescription( + "Password to be used when connecting to MongoDB. " + + "This is required only when MongoDB is configured to use authentication."); + + public static final Option> DATABASE = + Options.key("database") + .listType() + .noDefaultValue() + .withDescription("Name of the database to watch for changes."); + + public static final Option> COLLECTION = + Options.key("collection") + .listType() + .noDefaultValue() + .withDescription( + "Name of the collection in the database to watch for changes."); + + public static final Option CONNECTION_OPTIONS = + Options.key("connection.options") + .stringType() + .noDefaultValue() + .withDescription( + "The ampersand-separated MongoDB connection options. " + + "eg. replicaSet=test&connectTimeoutMS=300000"); + + public static final Option BATCH_SIZE = + Options.key("batch.size") + .intType() + .defaultValue(1024) + .withDescription("The cursor batch size. Defaults to 1024."); + + public static final Option POLL_MAX_BATCH_SIZE = + Options.key("poll.max.batch.size") + .intType() + .defaultValue(1024) + .withDescription( + "Maximum number of change stream documents " + + "to include in a single batch when polling for new data. " + + "This setting can be used to limit the amount of data buffered internally in the connector. " + + "Defaults to 1024."); + + public static final Option POLL_AWAIT_TIME_MILLIS = + Options.key("poll.await.time.ms") + .intType() + .defaultValue(1000) + .withDescription( + "The amount of time to wait before checking for new results on the change stream." + + "Defaults: 1000."); + + public static final Option HEARTBEAT_INTERVAL_MILLIS = + Options.key("heartbeat.interval.ms") + .intType() + .defaultValue(0) + .withDescription( + "The length of time in milliseconds between sending heartbeat messages." + + "Heartbeat messages contain the post batch resume token and are sent when no source records " + + "have been published in the specified interval. This improves the resumability of the connector " + + "for low volume namespaces. Use 0 to disable. Defaults to 0."); + + public static final Option INCREMENTAL_SNAPSHOT_CHUNK_SIZE_MB = + Options.key("incremental.snapshot.chunk.size.mb") + .intType() + .defaultValue(64) + .withDescription( + "The chunk size mb of incremental snapshot. Defaults to 64mb."); + + public static final Option> DEBEZIUM_PROPERTIES = + Options.key("debezium") + .mapType() + .defaultValue( + new HashMap() { + { + put("key.converter.schemas.enable", "false"); + put("value.converter.schemas.enable", "false"); + } + }) + .withDescription( + "Decides if the table options contains Debezium client properties that start with prefix 'debezium'."); + + public static final Option STARTUP_MODE = + Options.key(SourceOptions.STARTUP_MODE_KEY) + .singleChoice( + StartupMode.class, + Arrays.asList( + StartupMode.INITIAL, StartupMode.EARLIEST, StartupMode.LATEST)) + .defaultValue(StartupMode.INITIAL) + .withDescription( + "Optional startup mode for CDC source, valid enumerations are " + + "\"initial\", \"earliest\", \"latest\", \"timestamp\"\n or \"specific\""); + + public static final Option STOP_MODE = + Options.key(SourceOptions.STOP_MODE_KEY) + .singleChoice(StopMode.class, Collections.singletonList(StopMode.NEVER)) + .defaultValue(StopMode.NEVER) + .withDescription( + "Optional stop mode for CDC source, valid enumerations are " + + "\"never\", \"latest\", \"timestamp\"\n or \"specific\""); +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/exception/MongodbConnectorException.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/exception/MongodbConnectorException.java new file mode 100644 index 000000000000..2d2267e478cc --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/exception/MongodbConnectorException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception; + +import org.apache.seatunnel.common.exception.SeaTunnelErrorCode; +import org.apache.seatunnel.common.exception.SeaTunnelRuntimeException; + +public class MongodbConnectorException extends SeaTunnelRuntimeException { + + public MongodbConnectorException(SeaTunnelErrorCode seaTunnelErrorCode, String errorMessage) { + super(seaTunnelErrorCode, errorMessage); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/internal/MongodbClientProvider.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/internal/MongodbClientProvider.java new file mode 100644 index 000000000000..dc621ed9dc00 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/internal/MongodbClientProvider.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.internal; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; + +import com.mongodb.ConnectionString; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public enum MongodbClientProvider { + INSTANCE; + + private volatile MongoClient mongoClient; + + public MongoClient getOrCreateMongoClient(MongodbSourceConfig sourceConfig) { + if (mongoClient == null) { + ConnectionString connectionString = + new ConnectionString(sourceConfig.getConnectionString()); + log.info( + "Create and register mongo client {}@{}", + connectionString.getUsername(), + connectionString.getHosts()); + mongoClient = MongoClients.create(connectionString); + } + return mongoClient; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/MongoDBConnectorDeserializationSchema.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/MongoDBConnectorDeserializationSchema.java new file mode 100644 index 000000000000..f4cf4a503fb2 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/MongoDBConnectorDeserializationSchema.java @@ -0,0 +1,558 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.sender; + +import org.apache.seatunnel.api.source.Collector; +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.MapType; +import org.apache.seatunnel.api.table.type.RowKind; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.api.table.type.SqlType; +import org.apache.seatunnel.connectors.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonType; +import org.bson.BsonValue; +import org.bson.json.JsonMode; +import org.bson.json.JsonWriterSettings; +import org.bson.types.Decimal128; +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import com.mongodb.client.model.changestream.OperationType; + +import java.io.Serializable; +import java.lang.reflect.Array; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_DATA_TYPE; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_OPERATION; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DEFAULT_JSON_WRITER_SETTINGS; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ENCODE_VALUE_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.FULL_DOCUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.shade.com.google.common.base.Preconditions.checkNotNull; + +public class MongoDBConnectorDeserializationSchema + implements DebeziumDeserializationSchema { + + private final SeaTunnelDataType resultTypeInfo; + + private final DeserializationRuntimeConverter physicalConverter; + + public MongoDBConnectorDeserializationSchema( + SeaTunnelDataType physicalDataType, + SeaTunnelDataType resultTypeInfo) { + this.physicalConverter = createConverter(physicalDataType); + this.resultTypeInfo = resultTypeInfo; + } + + @Override + public void deserialize(@NotNull SourceRecord record, Collector out) { + Struct value = (Struct) record.value(); + Schema valueSchema = record.valueSchema(); + + OperationType op = operationTypeFor(record); + BsonDocument documentKey = + checkNotNull( + Objects.requireNonNull( + extractBsonDocument(value, valueSchema, DOCUMENT_KEY))); + BsonDocument fullDocument = extractBsonDocument(value, valueSchema, FULL_DOCUMENT); + + switch (op) { + case INSERT: + SeaTunnelRow insert = extractRowData(fullDocument); + insert.setRowKind(RowKind.INSERT); + emit(record, insert, out); + break; + case DELETE: + SeaTunnelRow delete = + new SeaTunnelRow( + new Object[] { + documentKey.get(ID_FIELD).asObjectId().getValue().toString() + }); + delete.setRowKind(RowKind.DELETE); + emit(record, delete, out); + break; + case UPDATE: + if (fullDocument == null) { + break; + } + SeaTunnelRow updateAfter = extractRowData(fullDocument); + updateAfter.setRowKind(RowKind.UPDATE_AFTER); + emit(record, updateAfter, out); + break; + case REPLACE: + SeaTunnelRow replaceAfter = extractRowData(fullDocument); + replaceAfter.setRowKind(RowKind.UPDATE_AFTER); + emit(record, replaceAfter, out); + break; + case INVALIDATE: + case DROP: + case DROP_DATABASE: + case RENAME: + case OTHER: + default: + break; + } + } + + @Override + public SeaTunnelDataType getProducedType() { + return resultTypeInfo; + } + + private @NotNull OperationType operationTypeFor(@NotNull SourceRecord record) { + Struct value = (Struct) record.value(); + return OperationType.fromString(value.getString("operationType")); + } + + // TODO:The dynamic schema will be completed based on this method later. + private void emit( + SourceRecord inRecord, + SeaTunnelRow physicalRow, + @NotNull Collector collector) { + collector.collect(physicalRow); + } + + private SeaTunnelRow extractRowData(BsonDocument document) { + checkNotNull(document); + return (SeaTunnelRow) physicalConverter.convert(document); + } + + private BsonDocument extractBsonDocument( + Struct value, @NotNull Schema valueSchema, String fieldName) { + if (valueSchema.field(fieldName) != null) { + String docString = value.getString(fieldName); + if (docString != null) { + return BsonDocument.parse(docString); + } + } + return null; + } + + // ------------------------------------------------------------------------------------- + // Runtime Converters + // ------------------------------------------------------------------------------------- + + @FunctionalInterface + public interface DeserializationRuntimeConverter extends Serializable { + Object convert(BsonValue bsonValue); + } + + public DeserializationRuntimeConverter createConverter(SeaTunnelDataType type) { + SerializableFunction internalRowConverter = + createNullSafeInternalConverter(type); + return new DeserializationRuntimeConverter() { + private static final long serialVersionUID = 1L; + + @Override + public Object convert(BsonValue bsonValue) { + return internalRowConverter.apply(bsonValue); + } + }; + } + + @Contract("_ -> new") + private static @NotNull SerializableFunction createNullSafeInternalConverter( + SeaTunnelDataType type) { + return wrapIntoNullSafeInternalConverter(createInternalConverter(type), type); + } + + @Contract(value = "_, _ -> new", pure = true) + private static @NotNull 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(@NotNull BsonValue bsonValue) { + return bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN(); + } + + private static SerializableFunction createInternalConverter( + @NotNull 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(@NotNull BsonValue bsonValue) { + Instant instant; + if (bsonValue.isTimestamp()) { + instant = Instant.ofEpochSecond(bsonValue.asTimestamp().getTime()); + } else if (bsonValue.isDateTime()) { + instant = Instant.ofEpochMilli(bsonValue.asDateTime().getValue()); + } else { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert to LocalDateTime from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + return Timestamp.from(instant).toLocalDateTime(); + } + + @SuppressWarnings("unchecked") + private static @NotNull SerializableFunction createRowConverter( + @NotNull SeaTunnelRowType type) { + SeaTunnelDataType[] fieldTypes = type.getFieldTypes(); + final SerializableFunction[] fieldConverters = + Arrays.stream(fieldTypes) + .map(MongoDBConnectorDeserializationSchema::createNullSafeInternalConverter) + .toArray(SerializableFunction[]::new); + int fieldCount = type.getTotalFields(); + + final String[] fieldNames = type.getFieldNames(); + + return new SerializableFunction() { + private static final long serialVersionUID = 1L; + + @Override + public Object apply(BsonValue bsonValue) { + if (!bsonValue.isDocument()) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert to rowType from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } + + BsonDocument document = bsonValue.asDocument(); + SeaTunnelRow row = new SeaTunnelRow(fieldCount); + for (int i = 0; i < fieldCount; i++) { + String fieldName = fieldNames[i]; + BsonValue fieldValue = document.get(fieldName); + Object convertedField = fieldConverters[i].apply(fieldValue); + row.setField(i, convertedField); + } + return row; + } + }; + } + + private static @NotNull SerializableFunction createArrayConverter( + @NotNull 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 @NotNull SerializableFunction createMapConverter( + String typeSummary, + @NotNull 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 @Nullable 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(@NotNull 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(@NotNull 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(@NotNull 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(@NotNull 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(@NotNull 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(@NotNull 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(@NotNull BsonValue bsonValue) { + if (bsonValue.isDecimal128()) { + Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value(); + if (decimal128Value.isFinite()) { + return bsonValue.asDecimal128().decimal128Value().bigDecimalValue(); + } else { + // DecimalData doesn't have the concept of infinity. + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert infinite bson decimal to Decimal type."); + } + } + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Unable to convert to decimal from unexpected value '" + + bsonValue + + "' of type " + + bsonValue.getBsonType()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/SerializableFunction.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/SerializableFunction.java new file mode 100644 index 000000000000..8905fbb02739 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/sender/SerializableFunction.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.sender; + +import java.io.Serializable; +import java.util.function.Function; + +@FunctionalInterface +public interface SerializableFunction extends Function, Serializable {} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java new file mode 100644 index 000000000000..86ea9a62381f --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/dialect/MongodbDialect.java @@ -0,0 +1,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.cdc.mongodb.source.dialect; + +import org.apache.seatunnel.connectors.cdc.base.dialect.DataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch.MongodbFetchTaskContext; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch.MongodbScanFetchTask; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch.MongodbStreamFetchTask; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamDescriptor; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters.MongodbChunkSplitter; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils; + +import org.bson.BsonDocument; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.client.MongoClient; +import io.debezium.relational.TableId; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DIALECT_NAME; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.collectionNames; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.collectionsFilter; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.databaseFilter; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.databaseNames; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.createMongoClient; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getChangeStreamDescriptor; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getCurrentClusterTime; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getLatestResumeToken; + +public class MongodbDialect implements DataSourceDialect { + + private final Map cache = + new ConcurrentHashMap<>(); + + @Override + public String getName() { + return DIALECT_NAME; + } + + @Override + public List discoverDataCollections(MongodbSourceConfig sourceConfig) { + CollectionDiscoveryUtils.CollectionDiscoveryInfo discoveryInfo = + discoverAndCacheDataCollections(sourceConfig); + return discoveryInfo.getDiscoveredCollections().stream() + .map(TableId::parse) + .collect(Collectors.toList()); + } + + @Override + public boolean isDataCollectionIdCaseSensitive(MongodbSourceConfig sourceConfig) { + // MongoDB's database names and collection names are case-sensitive. + return true; + } + + @Override + public ChunkSplitter createChunkSplitter(MongodbSourceConfig sourceConfig) { + return new MongodbChunkSplitter(sourceConfig); + } + + @Override + public FetchTask createFetchTask(@NotNull SourceSplitBase sourceSplitBase) { + if (sourceSplitBase.isSnapshotSplit()) { + return new MongodbScanFetchTask(sourceSplitBase.asSnapshotSplit()); + } else { + return new MongodbStreamFetchTask(sourceSplitBase.asIncrementalSplit()); + } + } + + @Override + public FetchTask.Context createFetchTaskContext( + SourceSplitBase sourceSplitBase, MongodbSourceConfig sourceConfig) { + CollectionDiscoveryUtils.CollectionDiscoveryInfo discoveryInfo = + discoverAndCacheDataCollections(sourceConfig); + ChangeStreamDescriptor changeStreamDescriptor = + getChangeStreamDescriptor( + sourceConfig, + discoveryInfo.getDiscoveredDatabases(), + discoveryInfo.getDiscoveredCollections()); + return new MongodbFetchTaskContext(this, sourceConfig, changeStreamDescriptor); + } + + private CollectionDiscoveryUtils.CollectionDiscoveryInfo discoverAndCacheDataCollections( + MongodbSourceConfig sourceConfig) { + return cache.computeIfAbsent( + sourceConfig, + config -> { + MongoClient mongoClient = createMongoClient(sourceConfig); + List discoveredDatabases = + databaseNames( + mongoClient, databaseFilter(sourceConfig.getDatabaseList())); + List discoveredCollections = + collectionNames( + mongoClient, + discoveredDatabases, + collectionsFilter(sourceConfig.getCollectionList())); + return new CollectionDiscoveryUtils.CollectionDiscoveryInfo( + discoveredDatabases, discoveredCollections); + }); + } + + public ChangeStreamOffset displayCurrentOffset(MongodbSourceConfig sourceConfig) { + MongoClient mongoClient = createMongoClient(sourceConfig); + CollectionDiscoveryUtils.CollectionDiscoveryInfo discoveryInfo = + discoverAndCacheDataCollections(sourceConfig); + ChangeStreamDescriptor changeStreamDescriptor = + getChangeStreamDescriptor( + sourceConfig, + discoveryInfo.getDiscoveredDatabases(), + discoveryInfo.getDiscoveredCollections()); + BsonDocument startupResumeToken = getLatestResumeToken(mongoClient, changeStreamDescriptor); + + ChangeStreamOffset changeStreamOffset; + if (startupResumeToken != null) { + changeStreamOffset = new ChangeStreamOffset(startupResumeToken); + } else { + changeStreamOffset = new ChangeStreamOffset(getCurrentClusterTime(mongoClient)); + } + + return changeStreamOffset; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbFetchTaskContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbFetchTaskContext.java new file mode 100644 index 000000000000..a0e2a79c8f4b --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbFetchTaskContext.java @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch; + +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.dialect.MongodbDialect; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamDescriptor; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils; + +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonType; +import org.bson.BsonValue; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.client.model.changestream.OperationType; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.pipeline.DataChangeEvent; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.util.LoggingContext; + +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.OPERATION_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_TRUE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SOURCE_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.TS_MS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.BsonUtils.compareBsonValue; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.getDocumentKey; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.getResumeToken; + +public class MongodbFetchTaskContext implements FetchTask.Context { + + private final MongodbDialect dialect; + private final MongodbSourceConfig sourceConfig; + private final ChangeStreamDescriptor changeStreamDescriptor; + private ChangeEventQueue changeEventQueue; + + public MongodbFetchTaskContext( + MongodbDialect dialect, + MongodbSourceConfig sourceConfig, + ChangeStreamDescriptor changeStreamDescriptor) { + this.dialect = dialect; + this.sourceConfig = sourceConfig; + this.changeStreamDescriptor = changeStreamDescriptor; + } + + public void configure(@NotNull SourceSplitBase sourceSplitBase) { + final int queueSize = + sourceSplitBase.isSnapshotSplit() ? Integer.MAX_VALUE : sourceConfig.getBatchSize(); + this.changeEventQueue = + new ChangeEventQueue.Builder() + .pollInterval(Duration.ofMillis(sourceConfig.getPollAwaitTimeMillis())) + .maxBatchSize(sourceConfig.getPollMaxBatchSize()) + .maxQueueSize(queueSize) + .loggingContextSupplier( + () -> + LoggingContext.forConnector( + "mongodb-cdc", + "mongodb-cdc-connector", + "mongodb-cdc-connector-task")) + .build(); + } + + public MongodbSourceConfig getSourceConfig() { + return sourceConfig; + } + + public MongodbDialect getDialect() { + return dialect; + } + + public ChangeStreamDescriptor getChangeStreamDescriptor() { + return changeStreamDescriptor; + } + + public ChangeEventQueue getQueue() { + return changeEventQueue; + } + + @Override + public TableId getTableId(SourceRecord record) { + return MongodbRecordUtils.getTableId(record); + } + + @Override + public Tables.TableFilter getTableFilter() { + // We have pushed down the filters to server side. + return Tables.TableFilter.includeAll(); + } + + @Override + public Offset getStreamOffset(SourceRecord record) { + return new ChangeStreamOffset(getResumeToken(record)); + } + + @Override + public boolean isDataChangeRecord(SourceRecord record) { + return MongodbRecordUtils.isDataChangeRecord(record); + } + + @Override + public boolean isRecordBetween( + SourceRecord record, @NotNull Object[] splitStart, @NotNull Object[] splitEnd) { + BsonDocument documentKey = getDocumentKey(record); + BsonDocument splitKeys = (BsonDocument) ((Object[]) splitStart[0])[0]; + String firstKey = splitKeys.getFirstKey(); + BsonValue keyValue = documentKey.get(firstKey); + BsonValue lowerBound = ((BsonDocument) ((Object[]) splitEnd[0])[1]).get(firstKey); + BsonValue upperBound = ((BsonDocument) ((Object[]) splitEnd[0])[1]).get(firstKey); + + if (isFullRange(lowerBound, upperBound)) { + return true; + } + + return isValueInRange(lowerBound, keyValue, upperBound); + } + + private boolean isFullRange(@NotNull BsonValue lowerBound, BsonValue upperBound) { + return lowerBound.getBsonType() == BsonType.MIN_KEY + && upperBound.getBsonType() == BsonType.MAX_KEY; + } + + private boolean isValueInRange(BsonValue lowerBound, BsonValue value, BsonValue upperBound) { + return compareBsonValue(lowerBound, value) <= 0 && compareBsonValue(value, upperBound) < 0; + } + + @Override + public void rewriteOutputBuffer( + Map outputBuffer, @NotNull SourceRecord changeRecord) { + Struct key = (Struct) changeRecord.key(); + Struct value = (Struct) changeRecord.value(); + + if (value != null) { + String operationType = value.getString(OPERATION_TYPE); + + switch (OperationType.fromString(operationType)) { + case INSERT: + case UPDATE: + case REPLACE: + outputBuffer.put(key, changeRecord); + break; + case DELETE: + outputBuffer.remove(key); + break; + default: + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Data change record meet UNKNOWN operation: " + operationType); + } + } + } + + @Override + public List formatMessageTimestamp( + @NotNull Collection snapshotRecords) { + return snapshotRecords.stream() + .peek( + record -> { + Struct value = (Struct) record.value(); + Struct source = new Struct(value.schema().field(SOURCE_FIELD).schema()); + source.put(TS_MS_FIELD, 0L); + source.put(SNAPSHOT_FIELD, SNAPSHOT_TRUE); + value.put(SOURCE_FIELD, source); + }) + .collect(Collectors.toList()); + } + + @Override + public void close() {} +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbScanFetchTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbScanFetchTask.java new file mode 100644 index 000000000000..effedaa0c9f0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbScanFetchTask.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch; + +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkEvent; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkKind; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.dialect.MongodbDialect; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils; + +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonInt64; +import org.bson.BsonString; +import org.bson.RawBsonDocument; +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.pipeline.DataChangeEvent; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.Collections; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.COLL_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DB_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.FULL_DOCUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.OPERATION_TYPE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.OPERATION_TYPE_INSERT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_TRUE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SOURCE_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.TS_MS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createSourceOffsetMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createWatermarkPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.createMongoClient; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getMongoCollection; + +@Slf4j +public class MongodbScanFetchTask implements FetchTask { + + private final SnapshotSplit snapshotSplit; + + private volatile boolean taskRunning = false; + + public MongodbScanFetchTask(SnapshotSplit snapshotSplit) { + this.snapshotSplit = snapshotSplit; + } + + @Override + public void execute(Context context) throws Exception { + MongodbFetchTaskContext taskContext = (MongodbFetchTaskContext) context; + MongodbSourceConfig sourceConfig = taskContext.getSourceConfig(); + MongodbDialect dialect = taskContext.getDialect(); + ChangeEventQueue changeEventQueue = taskContext.getQueue(); + taskRunning = true; + TableId collectionId = snapshotSplit.getTableId(); + final ChangeStreamOffset lowWatermark = dialect.displayCurrentOffset(sourceConfig); + log.info( + "Snapshot step 1 - Determining low watermark {} for split {}", + lowWatermark, + snapshotSplit); + changeEventQueue.enqueue( + new DataChangeEvent( + WatermarkEvent.create( + createWatermarkPartitionMap(collectionId.identifier()), + "__mongodb_watermarks", + snapshotSplit.splitId(), + WatermarkKind.LOW, + lowWatermark))); + + log.info("Snapshot step 2 - Snapshotting data"); + try (MongoCursor cursor = getSnapshotCursor(snapshotSplit, sourceConfig)) { + while (cursor.hasNext()) { + checkTaskRunning(); + BsonDocument valueDocument = normalizeSnapshotDocument(collectionId, cursor.next()); + BsonDocument keyDocument = new BsonDocument(ID_FIELD, valueDocument.get(ID_FIELD)); + + SourceRecord snapshotRecord = + buildSourceRecord(sourceConfig, collectionId, keyDocument, valueDocument); + + changeEventQueue.enqueue(new DataChangeEvent(snapshotRecord)); + } + + ChangeStreamOffset highWatermark = dialect.displayCurrentOffset(sourceConfig); + log.info( + "Snapshot step 3 - Determining high watermark {} for split {}", + highWatermark, + snapshotSplit); + changeEventQueue.enqueue( + new DataChangeEvent( + WatermarkEvent.create( + createWatermarkPartitionMap(collectionId.identifier()), + "__mongodb_watermarks", + snapshotSplit.splitId(), + WatermarkKind.HIGH, + highWatermark))); + + log.info( + "Snapshot step 4 - Back fill stream split for snapshot split {}", + snapshotSplit); + final IncrementalSplit dataBackfillSplit = + createBackfillStreamSplit(lowWatermark, highWatermark); + final boolean streamBackfillRequired = + dataBackfillSplit.getStopOffset().isAfter(dataBackfillSplit.getStartupOffset()); + + if (!streamBackfillRequired) { + changeEventQueue.enqueue( + new DataChangeEvent( + WatermarkEvent.create( + createWatermarkPartitionMap(collectionId.identifier()), + "__mongodb_watermarks", + dataBackfillSplit.splitId(), + WatermarkKind.END, + dataBackfillSplit.getStopOffset()))); + } else { + MongodbStreamFetchTask dataBackfillTask = + new MongodbStreamFetchTask(dataBackfillSplit); + dataBackfillTask.execute(taskContext); + } + } catch (Exception e) { + log.error( + String.format( + "Execute snapshot read subtask for mongo split %s fail", snapshotSplit), + e); + throw e; + } finally { + taskRunning = false; + } + } + + @NotNull private MongoCursor getSnapshotCursor( + @NotNull SnapshotSplit snapshotSplit, MongodbSourceConfig sourceConfig) { + MongoClient mongoClient = createMongoClient(sourceConfig); + MongoCollection collection = + getMongoCollection(mongoClient, snapshotSplit.getTableId(), RawBsonDocument.class); + BsonDocument startKey = (BsonDocument) snapshotSplit.getSplitStart()[1]; + BsonDocument endKey = (BsonDocument) snapshotSplit.getSplitEnd()[1]; + BsonDocument hint = (BsonDocument) snapshotSplit.getSplitStart()[0]; + + return collection + .find() + .min(startKey) + .max(endKey) + .hint(hint) + .batchSize(sourceConfig.getBatchSize()) + .noCursorTimeout(true) + .cursor(); + } + + @NotNull private SourceRecord buildSourceRecord( + @NotNull MongodbSourceConfig sourceConfig, + @NotNull TableId collectionId, + BsonDocument keyDocument, + BsonDocument valueDocument) { + return MongodbRecordUtils.buildSourceRecord( + createPartitionMap( + sourceConfig.getHosts(), collectionId.catalog(), collectionId.table()), + createSourceOffsetMap(keyDocument.getDocument(ID_FIELD), true), + collectionId.identifier(), + keyDocument, + valueDocument); + } + + private void checkTaskRunning() { + if (!taskRunning) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Interrupted while snapshotting collection"); + } + } + + @Override + public boolean isRunning() { + return taskRunning; + } + + @Override + public void shutdown() {} + + @Override + public SnapshotSplit getSplit() { + return snapshotSplit; + } + + @NotNull @Contract("_, _ -> new") + private IncrementalSplit createBackfillStreamSplit( + ChangeStreamOffset lowWatermark, ChangeStreamOffset highWatermark) { + return new IncrementalSplit( + snapshotSplit.splitId(), + Collections.singletonList(snapshotSplit.getTableId()), + lowWatermark, + highWatermark, + new ArrayList<>()); + } + + private BsonDocument normalizeSnapshotDocument( + @NotNull final TableId collectionId, @NotNull final BsonDocument originalDocument) { + return new BsonDocument() + .append(ID_FIELD, new BsonDocument(ID_FIELD, originalDocument.get(ID_FIELD))) + .append(OPERATION_TYPE, new BsonString(OPERATION_TYPE_INSERT)) + .append( + NS_FIELD, + new BsonDocument(DB_FIELD, new BsonString(collectionId.catalog())) + .append(COLL_FIELD, new BsonString(collectionId.table()))) + .append(DOCUMENT_KEY, new BsonDocument(ID_FIELD, originalDocument.get(ID_FIELD))) + .append(FULL_DOCUMENT, originalDocument) + .append(TS_MS_FIELD, new BsonInt64(System.currentTimeMillis())) + .append( + SOURCE_FIELD, + new BsonDocument(SNAPSHOT_FIELD, new BsonString(SNAPSHOT_TRUE)) + .append(TS_MS_FIELD, new BsonInt64(0L))); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbStreamFetchTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbStreamFetchTask.java new file mode 100644 index 000000000000..eaacfa1cb5b0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/fetch/MongodbStreamFetchTask.java @@ -0,0 +1,356 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.fetch; + +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkEvent; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkKind; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamDescriptor; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils; + +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonInt64; +import org.bson.BsonString; +import org.bson.BsonTimestamp; +import org.bson.Document; +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.MongoCommandException; +import com.mongodb.MongoNamespace; +import com.mongodb.client.ChangeStreamIterable; +import com.mongodb.client.MongoChangeStreamCursor; +import com.mongodb.client.MongoClient; +import com.mongodb.kafka.connect.source.heartbeat.HeartbeatManager; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.pipeline.DataChangeEvent; +import lombok.extern.slf4j.Slf4j; + +import javax.annotation.Nullable; + +import java.time.Instant; +import java.util.Optional; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_OPERATION; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.CLUSTER_TIME_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.COLL_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DB_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.FAILED_TO_PARSE_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.FALSE_FALSE; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ILLEGAL_OPERATION_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SNAPSHOT_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SOURCE_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.TS_MS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UNAUTHORIZED_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UNKNOWN_FIELD_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamOffset.NO_STOPPING_OFFSET; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createHeartbeatPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createSourceOffsetMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.createWatermarkPartitionMap; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.currentBsonTimestamp; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.getResumeToken; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.createMongoClient; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getChangeStreamIterable; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getCurrentClusterTime; + +@Slf4j +public class MongodbStreamFetchTask implements FetchTask { + + private final IncrementalSplit streamSplit; + private volatile boolean taskRunning = false; + + private MongodbSourceConfig sourceConfig; + private final Time time = new SystemTime(); + private boolean supportsStartAtOperationTime = true; + private boolean supportsStartAfter = true; + + public MongodbStreamFetchTask(IncrementalSplit streamSplit) { + this.streamSplit = streamSplit; + } + + @Override + public void execute(Context context) throws Exception { + MongodbFetchTaskContext taskContext = (MongodbFetchTaskContext) context; + this.sourceConfig = taskContext.getSourceConfig(); + + ChangeStreamDescriptor descriptor = taskContext.getChangeStreamDescriptor(); + ChangeEventQueue queue = taskContext.getQueue(); + + MongoClient mongoClient = createMongoClient(sourceConfig); + MongoChangeStreamCursor changeStreamCursor = + openChangeStreamCursor(descriptor); + HeartbeatManager heartbeatManager = openHeartbeatManagerIfNeeded(changeStreamCursor); + + final long startPoll = time.milliseconds(); + long nextUpdate = startPoll + sourceConfig.getPollAwaitTimeMillis(); + this.taskRunning = true; + try { + while (taskRunning) { + Optional next = Optional.ofNullable(changeStreamCursor.tryNext()); + SourceRecord changeRecord = null; + if (!next.isPresent()) { + long untilNext = nextUpdate - time.milliseconds(); + if (untilNext > 0) { + log.debug("Waiting {} ms to poll change records", untilNext); + time.sleep(untilNext); + continue; + } + + if (heartbeatManager != null) { + changeRecord = + heartbeatManager + .heartbeat() + .map(this::normalizeHeartbeatRecord) + .orElse(null); + } + // update nextUpdateTime + nextUpdate = time.milliseconds() + sourceConfig.getPollAwaitTimeMillis(); + } else { + BsonDocument changeStreamDocument = next.get(); + MongoNamespace namespace = getMongoNamespace(changeStreamDocument); + + BsonDocument resumeToken = changeStreamDocument.getDocument(ID_FIELD); + BsonDocument valueDocument = + normalizeChangeStreamDocument(changeStreamDocument); + + log.trace("Adding {} to {}", valueDocument, namespace.getFullName()); + + changeRecord = + MongodbRecordUtils.buildSourceRecord( + createPartitionMap( + sourceConfig.getHosts(), + namespace.getDatabaseName(), + namespace.getCollectionName()), + createSourceOffsetMap(resumeToken, false), + namespace.getFullName(), + changeStreamDocument.getDocument(ID_FIELD), + valueDocument); + } + + if (changeRecord != null) { + queue.enqueue(new DataChangeEvent(changeRecord)); + } + + if (isBoundedRead()) { + ChangeStreamOffset currentOffset; + if (changeRecord != null) { + currentOffset = new ChangeStreamOffset(getResumeToken(changeRecord)); + } else { + // Heartbeat is not turned on or there is no update event + currentOffset = new ChangeStreamOffset(getCurrentClusterTime(mongoClient)); + } + + // Reach the high watermark, the binlog fetcher should be finished + if (currentOffset.isAtOrAfter(streamSplit.getStopOffset())) { + // send watermark end event + SourceRecord watermark = + WatermarkEvent.create( + createWatermarkPartitionMap(descriptor.toString()), + "__mongodb_watermarks", + streamSplit.splitId(), + WatermarkKind.END, + currentOffset); + + queue.enqueue(new DataChangeEvent(watermark)); + break; + } + } + } + } catch (Exception e) { + log.error("Poll change stream records failed ", e); + throw e; + } finally { + taskRunning = false; + if (changeStreamCursor != null) { + changeStreamCursor.close(); + } + } + } + + @Override + public boolean isRunning() { + return taskRunning; + } + + @Override + public void shutdown() {} + + @Override + public IncrementalSplit getSplit() { + return streamSplit; + } + + private MongoChangeStreamCursor openChangeStreamCursor( + ChangeStreamDescriptor changeStreamDescriptor) { + ChangeStreamOffset offset = + new ChangeStreamOffset(streamSplit.getStartupOffset().getOffset()); + + ChangeStreamIterable changeStreamIterable = + getChangeStreamIterable(sourceConfig, changeStreamDescriptor); + + BsonDocument resumeToken = offset.getResumeToken(); + BsonTimestamp timestamp = offset.getTimestamp(); + + if (resumeToken != null) { + if (supportsStartAfter) { + log.info("Open the change stream after the previous offset: {}", resumeToken); + changeStreamIterable.startAfter(resumeToken); + } else { + log.info( + "Open the change stream after the previous offset using resumeAfter: {}", + resumeToken); + changeStreamIterable.resumeAfter(resumeToken); + } + } else { + if (supportsStartAtOperationTime) { + log.info("Open the change stream at the timestamp: {}", timestamp); + changeStreamIterable.startAtOperationTime(timestamp); + } else { + log.warn("Open the change stream of the latest offset"); + } + } + + try { + return (MongoChangeStreamCursor) + changeStreamIterable.withDocumentClass(BsonDocument.class).cursor(); + } catch (MongoCommandException e) { + if (e.getErrorCode() == FAILED_TO_PARSE_ERROR + || e.getErrorCode() == UNKNOWN_FIELD_ERROR) { + if (e.getErrorMessage().contains("startAtOperationTime")) { + supportsStartAtOperationTime = false; + return openChangeStreamCursor(changeStreamDescriptor); + } else if (e.getErrorMessage().contains("startAfter")) { + supportsStartAfter = false; + return openChangeStreamCursor(changeStreamDescriptor); + } else { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Open change stream failed"); + } + } else if (e.getErrorCode() == ILLEGAL_OPERATION_ERROR) { + throw new MongodbConnectorException( + UNSUPPORTED_OPERATION, + String.format( + "Illegal $changeStream operation: %s %s", + e.getErrorMessage(), e.getErrorCode())); + + } else if (e.getErrorCode() == UNAUTHORIZED_ERROR) { + throw new MongodbConnectorException( + UNSUPPORTED_OPERATION, + String.format( + "Unauthorized $changeStream operation: %s %s", + e.getErrorMessage(), e.getErrorCode())); + + } else { + throw new MongodbConnectorException(ILLEGAL_ARGUMENT, "Open change stream failed"); + } + } + } + + @Nullable private HeartbeatManager openHeartbeatManagerIfNeeded( + MongoChangeStreamCursor changeStreamCursor) { + if (sourceConfig.getHeartbeatIntervalMillis() > 0) { + return new HeartbeatManager( + time, + changeStreamCursor, + sourceConfig.getHeartbeatIntervalMillis(), + "__mongodb_heartbeats", + createHeartbeatPartitionMap(sourceConfig.getHosts())); + } + return null; + } + + @NotNull @Contract("_ -> param1") + private BsonDocument normalizeChangeStreamDocument(@NotNull BsonDocument changeStreamDocument) { + // _id: primary key of change document. + BsonDocument normalizedDocument = normalizeKeyDocument(changeStreamDocument); + changeStreamDocument.put(ID_FIELD, normalizedDocument); + + // ts_ms: It indicates the time at which the reader processed the event. + changeStreamDocument.put(TS_MS_FIELD, new BsonInt64(System.currentTimeMillis())); + + // source + BsonDocument source = new BsonDocument(); + source.put(SNAPSHOT_FIELD, new BsonString(FALSE_FALSE)); + + if (!changeStreamDocument.containsKey(CLUSTER_TIME_FIELD)) { + log.warn( + "Cannot extract clusterTime from change stream event, fallback to current timestamp."); + changeStreamDocument.put(CLUSTER_TIME_FIELD, currentBsonTimestamp()); + } + + // source.ts_ms + // It indicates the time that the change was made in the database. If the record is read + // from snapshot of the table instead of the change stream, the value is always 0. + BsonTimestamp clusterTime = changeStreamDocument.getTimestamp(CLUSTER_TIME_FIELD); + Instant clusterInstant = Instant.ofEpochSecond(clusterTime.getTime()); + source.put(TS_MS_FIELD, new BsonInt64(clusterInstant.toEpochMilli())); + changeStreamDocument.put(SOURCE_FIELD, source); + + return changeStreamDocument; + } + + @NotNull private BsonDocument normalizeKeyDocument(@NotNull BsonDocument changeStreamDocument) { + BsonDocument documentKey = changeStreamDocument.getDocument(DOCUMENT_KEY); + BsonDocument primaryKey = new BsonDocument(ID_FIELD, documentKey.get(ID_FIELD)); + return new BsonDocument(ID_FIELD, primaryKey); + } + + @NotNull @Contract("_ -> new") + private SourceRecord normalizeHeartbeatRecord(@NotNull SourceRecord heartbeatRecord) { + final Struct heartbeatValue = + new Struct(SchemaBuilder.struct().field(TS_MS_FIELD, Schema.INT64_SCHEMA).build()); + heartbeatValue.put(TS_MS_FIELD, Instant.now().toEpochMilli()); + + return new SourceRecord( + heartbeatRecord.sourcePartition(), + heartbeatRecord.sourceOffset(), + heartbeatRecord.topic(), + heartbeatRecord.keySchema(), + heartbeatRecord.key(), + SchemaBuilder.struct().field(TS_MS_FIELD, Schema.INT64_SCHEMA).build(), + heartbeatValue); + } + + @NotNull private MongoNamespace getMongoNamespace(@NotNull BsonDocument changeStreamDocument) { + BsonDocument ns = changeStreamDocument.getDocument(NS_FIELD); + + return new MongoNamespace( + ns.getString(DB_FIELD).getValue(), ns.getString(COLL_FIELD).getValue()); + } + + private boolean isBoundedRead() { + return !NO_STOPPING_OFFSET.equals(streamSplit.getStopOffset()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamDescriptor.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamDescriptor.java new file mode 100644 index 000000000000..96b5064262cb --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamDescriptor.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset; + +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; + +import io.debezium.relational.TableId; +import lombok.AllArgsConstructor; +import lombok.Getter; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.regex.Pattern; + +@AllArgsConstructor +@Getter +public class ChangeStreamDescriptor implements Serializable { + private static final long serialVersionUID = 1L; + + @Nullable private final String database; + @Nullable private final String collection; + @Nullable private final Pattern databaseRegex; + @Nullable private final Pattern namespaceRegex; + + @NotNull @Contract("_ -> new") + public static ChangeStreamDescriptor collection(@NotNull TableId collectionId) { + return collection(collectionId.catalog(), collectionId.table()); + } + + @NotNull @Contract("_, _ -> new") + public static ChangeStreamDescriptor collection(String database, String collection) { + return new ChangeStreamDescriptor(database, collection, null, null); + } + + @NotNull @Contract("_ -> new") + public static ChangeStreamDescriptor database(String database) { + return new ChangeStreamDescriptor(database, null, null, null); + } + + @NotNull @Contract("_, _ -> new") + public static ChangeStreamDescriptor database(String database, Pattern namespaceRegex) { + return new ChangeStreamDescriptor(database, null, null, namespaceRegex); + } + + @NotNull @Contract("_ -> new") + public static ChangeStreamDescriptor deployment(Pattern databaseRegex) { + return new ChangeStreamDescriptor(null, null, databaseRegex, null); + } + + @NotNull @Contract("_, _ -> new") + public static ChangeStreamDescriptor deployment(Pattern databaseRegex, Pattern namespaceRegex) { + return new ChangeStreamDescriptor(null, null, databaseRegex, namespaceRegex); + } + + @NotNull @Contract(" -> new") + public static ChangeStreamDescriptor deployment() { + return new ChangeStreamDescriptor(null, null, null, null); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffset.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffset.java new file mode 100644 index 000000000000..35acf43bbace --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffset.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset; + +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; + +import org.bson.BsonDocument; +import org.bson.BsonTimestamp; + +import javax.annotation.Nullable; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.RESUME_TOKEN_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.TIMESTAMP_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.maximumBsonTimestamp; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ResumeToken.decodeTimestamp; + +public class ChangeStreamOffset extends Offset { + + private static final long serialVersionUID = 1L; + + public static final ChangeStreamOffset NO_STOPPING_OFFSET = + new ChangeStreamOffset(maximumBsonTimestamp()); + + public ChangeStreamOffset(Map offset) { + this.offset = offset; + } + + public ChangeStreamOffset(BsonDocument resumeToken) { + Objects.requireNonNull(resumeToken); + Map offsetMap = new HashMap<>(); + offsetMap.put(TIMESTAMP_FIELD, String.valueOf(decodeTimestamp(resumeToken).getValue())); + offsetMap.put(RESUME_TOKEN_FIELD, resumeToken.toJson()); + this.offset = offsetMap; + } + + public ChangeStreamOffset(BsonTimestamp timestamp) { + Objects.requireNonNull(timestamp); + Map offsetMap = new HashMap<>(); + offsetMap.put(TIMESTAMP_FIELD, String.valueOf(timestamp.getValue())); + offsetMap.put(RESUME_TOKEN_FIELD, null); + this.offset = offsetMap; + } + + @Nullable public BsonDocument getResumeToken() { + String resumeTokenJson = offset.get(RESUME_TOKEN_FIELD); + return Optional.ofNullable(resumeTokenJson).map(BsonDocument::parse).orElse(null); + } + + public BsonTimestamp getTimestamp() { + long timestamp = Long.parseLong(offset.get(TIMESTAMP_FIELD)); + return new BsonTimestamp(timestamp); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof ChangeStreamOffset)) { + return false; + } + ChangeStreamOffset that = (ChangeStreamOffset) o; + return offset.equals(that.offset); + } + + @Override + public int compareTo(Offset offset) { + if (offset == null) { + return -1; + } + ChangeStreamOffset that = (ChangeStreamOffset) offset; + return this.getTimestamp().compareTo(that.getTimestamp()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffsetFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffsetFactory.java new file mode 100644 index 000000000000..c53d92aa67b9 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/offset/ChangeStreamOffsetFactory.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset; + +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; +import org.apache.seatunnel.connectors.cdc.base.source.offset.OffsetFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import java.util.Map; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.UNSUPPORTED_OPERATION; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.bsonTimestampFromEpochMillis; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbRecordUtils.currentBsonTimestamp; + +public class ChangeStreamOffsetFactory extends OffsetFactory { + + @Override + public Offset earliest() { + return new ChangeStreamOffset(currentBsonTimestamp()); + } + + @Override + public Offset neverStop() { + return ChangeStreamOffset.NO_STOPPING_OFFSET; + } + + @Override + public Offset latest() { + return new ChangeStreamOffset(currentBsonTimestamp()); + } + + @Override + public Offset specific(Map offset) { + return new ChangeStreamOffset(offset); + } + + @Override + public Offset specific(String filename, Long position) { + throw new MongodbConnectorException( + UNSUPPORTED_OPERATION, "not supported create new Offset by filename and position."); + } + + @Override + public Offset timestamp(long timestamp) { + return new ChangeStreamOffset(bsonTimestampFromEpochMillis(timestamp)); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/MongodbChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/MongodbChunkSplitter.java new file mode 100644 index 000000000000..18549c3f4b87 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/MongodbChunkSplitter.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; + +import io.debezium.relational.TableId; + +import java.util.Collection; + +public class MongodbChunkSplitter implements ChunkSplitter { + + private final MongodbSourceConfig sourceConfig; + + public MongodbChunkSplitter(MongodbSourceConfig sourceConfig) { + this.sourceConfig = sourceConfig; + } + + @Override + public Collection generateSplits(TableId collectionId) { + SplitContext splitContext = SplitContext.of(sourceConfig, collectionId); + SplitStrategy splitStrategy = + splitContext.isShardedCollection() + ? ShardedSplitStrategy.INSTANCE + : SplitVectorSplitStrategy.INSTANCE; + return splitStrategy.split(splitContext); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SampleBucketSplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SampleBucketSplitStrategy.java new file mode 100644 index 000000000000..180775908775 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SampleBucketSplitStrategy.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.bson.BsonDocument; +import org.bson.BsonValue; +import org.bson.conversions.Bson; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.client.MongoCollection; +import io.debezium.relational.TableId; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import static com.mongodb.client.model.Aggregates.bucketAuto; +import static com.mongodb.client.model.Aggregates.sample; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MAX_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MIN_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.boundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.maxUpperBoundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.minLowerBoundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.getMongoCollection; + +public enum SampleBucketSplitStrategy implements SplitStrategy { + INSTANCE; + + private static final int DEFAULT_SAMPLING_THRESHOLD = 102400; + + private static final double DEFAULT_SAMPLING_RATE = 0.05; + + @NotNull @Override + public Collection split(@NotNull SplitContext splitContext) { + long chunkSizeInBytes = (long) splitContext.getChunkSizeMB() * 1024 * 1024; + + long sizeInBytes = splitContext.getSizeInBytes(); + long count = splitContext.getDocumentCount(); + + // If collection's total uncompressed size less than chunk size, + // treat the entire collection as single chunk. + if (sizeInBytes < chunkSizeInBytes) { + return SingleSplitStrategy.INSTANCE.split(splitContext); + } + + int numChunks = (int) (sizeInBytes / chunkSizeInBytes) + 1; + int numberOfSamples; + if (count < DEFAULT_SAMPLING_THRESHOLD) { + // full sampling if document count less than sampling size threshold. + numberOfSamples = (int) count; + } else { + // sampled using sample rate. + numberOfSamples = (int) Math.floor(count * DEFAULT_SAMPLING_RATE); + } + + TableId collectionId = splitContext.getCollectionId(); + + MongoCollection collection = + getMongoCollection(splitContext.getMongoClient(), collectionId, BsonDocument.class); + + List pipeline = new ArrayList<>(); + if (numberOfSamples != count) { + pipeline.add(sample(numberOfSamples)); + } + pipeline.add(bucketAuto("$" + ID_FIELD, numChunks)); + + List chunks = + collection.aggregate(pipeline).allowDiskUse(true).into(new ArrayList<>()); + + SeaTunnelRowType rowType = shardKeysToRowType(Collections.singleton(ID_FIELD)); + + List snapshotSplits = new ArrayList<>(chunks.size() + 2); + + SnapshotSplit firstSplit = + new SnapshotSplit( + splitId(collectionId, 0), + collectionId, + rowType, + minLowerBoundOfId(), + boundOfId(lowerBoundOfBucket(chunks.get(0))), + null); + snapshotSplits.add(firstSplit); + + for (int i = 0; i < chunks.size(); i++) { + BsonDocument bucket = chunks.get(i); + snapshotSplits.add( + new SnapshotSplit( + splitId(collectionId, i + 1), + collectionId, + rowType, + boundOfId(lowerBoundOfBucket(bucket)), + boundOfId(upperBoundOfBucket(bucket)), + null)); + } + + SnapshotSplit lastSplit = + new SnapshotSplit( + splitId(collectionId, chunks.size() + 1), + collectionId, + rowType, + boundOfId(upperBoundOfBucket(chunks.get(chunks.size() - 1))), + maxUpperBoundOfId(), + null); + snapshotSplits.add(lastSplit); + + return snapshotSplits; + } + + private BsonDocument bucketBounds(@NotNull BsonDocument bucket) { + return bucket.getDocument(ID_FIELD); + } + + private BsonValue lowerBoundOfBucket(BsonDocument bucket) { + return bucketBounds(bucket).get(MIN_FIELD); + } + + private BsonValue upperBoundOfBucket(BsonDocument bucket) { + return bucketBounds(bucket).get(MAX_FIELD); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/ShardedSplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/ShardedSplitStrategy.java new file mode 100644 index 000000000000..fc93c9f8e28e --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/ShardedSplitStrategy.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.bson.BsonBoolean; +import org.bson.BsonDocument; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.MongoQueryException; +import com.mongodb.client.MongoClient; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DROPPED_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MAX_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MIN_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UNAUTHORIZED_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.readChunks; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.readCollectionMetadata; + +@Slf4j +public class ShardedSplitStrategy implements SplitStrategy { + + public static final ShardedSplitStrategy INSTANCE = new ShardedSplitStrategy(); + + private ShardedSplitStrategy() {} + + @Override + public Collection split(@NotNull SplitContext splitContext) { + TableId collectionId = splitContext.getCollectionId(); + MongoClient mongoClient = splitContext.getMongoClient(); + + List chunks; + BsonDocument collectionMetadata; + try { + collectionMetadata = readCollectionMetadata(mongoClient, collectionId); + if (!isValidShardedCollection(collectionMetadata)) { + log.warn( + "Collection {} does not appear to be sharded, fallback to SampleSplitter.", + collectionId); + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + chunks = readChunks(mongoClient, collectionMetadata); + } catch (MongoQueryException e) { + if (e.getErrorCode() == UNAUTHORIZED_ERROR) { + log.warn( + "Unauthorized to read config.collections or config.chunks: {}, fallback to SampleSplitter.", + e.getErrorMessage()); + } else { + log.warn( + "Read config.chunks collection failed: {}, fallback to SampleSplitter", + e.getErrorMessage()); + } + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + + if (chunks.isEmpty()) { + log.warn( + "Collection {} does not appear to be sharded, fallback to SampleSplitter.", + collectionId); + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + + BsonDocument splitKeys = collectionMetadata.getDocument("key"); + SeaTunnelRowType rowType = shardKeysToRowType(splitKeys); + + List snapshotSplits = new ArrayList<>(chunks.size()); + for (int i = 0; i < chunks.size(); i++) { + BsonDocument chunk = chunks.get(i); + snapshotSplits.add( + new SnapshotSplit( + splitId(collectionId, i), + collectionId, + rowType, + new Object[] {splitKeys, chunk.getDocument(MIN_FIELD)}, + new Object[] {splitKeys, chunk.getDocument(MAX_FIELD)}, + null)); + } + + return snapshotSplits; + } + + private boolean isValidShardedCollection(BsonDocument collectionMetadata) { + return collectionMetadata != null + && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue(); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SingleSplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SingleSplitStrategy.java new file mode 100644 index 000000000000..7826ba1b2217 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SingleSplitStrategy.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.jetbrains.annotations.NotNull; + +import io.debezium.relational.TableId; + +import java.util.Collection; +import java.util.Collections; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.maxUpperBoundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.minLowerBoundOfId; + +public enum SingleSplitStrategy implements SplitStrategy { + INSTANCE; + + @Override + public Collection split(@NotNull SplitContext splitContext) { + TableId collectionId = splitContext.getCollectionId(); + SnapshotSplit snapshotSplit = createSnapshotSplit(collectionId); + return Collections.singletonList(snapshotSplit); + } + + @NotNull private SnapshotSplit createSnapshotSplit(TableId collectionId) { + SeaTunnelRowType rowType = shardKeysToRowType(Collections.singleton(ID_FIELD)); + return new SnapshotSplit( + splitId(collectionId, 0), + collectionId, + rowType, + minLowerBoundOfId(), + maxUpperBoundOfId(), + null); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitContext.java new file mode 100644 index 000000000000..bed9b1787a24 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitContext.java @@ -0,0 +1,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. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils; + +import org.bson.BsonBoolean; +import org.bson.BsonDocument; +import org.bson.BsonInt64; +import org.bson.BsonNumber; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.client.MongoClient; +import io.debezium.relational.TableId; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.collStats; + +public class SplitContext { + + private final MongoClient mongoClient; + private final TableId collectionId; + private final BsonDocument collectionStats; + private final int chunkSizeMB; + + public SplitContext( + MongoClient mongoClient, + TableId collectionId, + BsonDocument collectionStats, + int chunkSizeMB) { + this.mongoClient = mongoClient; + this.collectionId = collectionId; + this.collectionStats = collectionStats; + this.chunkSizeMB = chunkSizeMB; + } + + @NotNull public static SplitContext of(MongodbSourceConfig sourceConfig, TableId collectionId) { + MongoClient mongoClient = MongodbUtils.createMongoClient(sourceConfig); + BsonDocument collectionStats = collStats(mongoClient, collectionId); + int chunkSizeMB = sourceConfig.getSplitSize(); + return new SplitContext(mongoClient, collectionId, collectionStats, chunkSizeMB); + } + + public MongoClient getMongoClient() { + return mongoClient; + } + + public TableId getCollectionId() { + return collectionId; + } + + public int getChunkSizeMB() { + return chunkSizeMB; + } + + public long getDocumentCount() { + return getNumberValue(collectionStats, "count"); + } + + public long getSizeInBytes() { + return getNumberValue(collectionStats, "size"); + } + + public long getAvgObjSizeInBytes() { + return getNumberValue(collectionStats, "avgObjSize"); + } + + public boolean isShardedCollection() { + return collectionStats.getBoolean("sharded", BsonBoolean.FALSE).getValue(); + } + + private long getNumberValue(@NotNull BsonDocument document, String fieldName) { + BsonNumber number = document.getNumber(fieldName, new BsonInt64(0)); + return number.longValue(); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitStrategy.java new file mode 100644 index 000000000000..6d093bd9419a --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitStrategy.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.bson.BsonDocument; +import org.jetbrains.annotations.NotNull; + +import io.debezium.relational.TableId; + +import java.util.Collection; + +import static org.apache.seatunnel.api.table.type.BasicType.INT_TYPE; + +public interface SplitStrategy { + + Collection split(SplitContext splitContext); + + default String splitId(@NotNull TableId collectionId, int chunkId) { + return String.format("%s:%d", collectionId.identifier(), chunkId); + } + + default SeaTunnelRowType shardKeysToRowType(@NotNull BsonDocument shardKeys) { + return shardKeysToRowType(shardKeys.keySet()); + } + + default SeaTunnelRowType shardKeysToRowType(@NotNull Collection shardKeys) { + SeaTunnelDataType[] fieldTypes = + shardKeys.stream() + // We cannot get the exact type of the shard key, only the ordering of the + // shard index. + // Use the INT type as a placeholder. + .map(key -> INT_TYPE) + .toArray(SeaTunnelDataType[]::new); + String[] fieldNames = shardKeys.toArray(new String[0]); + return new SeaTunnelRowType(fieldNames, fieldTypes); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitVectorSplitStrategy.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitVectorSplitStrategy.java new file mode 100644 index 000000000000..749682e3370f --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/source/splitters/SplitVectorSplitStrategy.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.splitters; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; + +import org.apache.commons.collections4.CollectionUtils; + +import org.bson.BsonArray; +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonMinKey; +import org.bson.BsonValue; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.MongoCommandException; +import com.mongodb.client.MongoClient; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UNAUTHORIZED_ERROR; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.boundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.ChunkUtils.maxUpperBoundOfId; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.isCommandSucceed; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.MongodbUtils.splitVector; + +@Slf4j +public enum SplitVectorSplitStrategy implements SplitStrategy { + INSTANCE; + + @Override + public Collection split(@NotNull SplitContext splitContext) { + MongoClient mongoClient = splitContext.getMongoClient(); + TableId collectionId = splitContext.getCollectionId(); + int chunkSizeMB = splitContext.getChunkSizeMB(); + + BsonDocument keyPattern = new BsonDocument(ID_FIELD, new BsonInt32(1)); + + BsonDocument splitResult; + try { + splitResult = splitVector(mongoClient, collectionId, keyPattern, chunkSizeMB); + } catch (MongoCommandException e) { + if (e.getErrorCode() == UNAUTHORIZED_ERROR) { + log.warn( + "Unauthorized to execute splitVector command: {}, fallback to SampleSplitter", + e.getErrorMessage()); + } else { + log.warn( + "Execute splitVector command failed: {}, fallback to SampleSplitter", + e.getErrorMessage()); + } + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + + if (!isCommandSucceed(splitResult)) { + log.warn( + "Could not calculate standalone splits: {}, fallback to SampleSplitter", + splitResult.getString("errmsg")); + return SampleBucketSplitStrategy.INSTANCE.split(splitContext); + } + + BsonArray splitKeys = splitResult.getArray("splitKeys"); + if (CollectionUtils.isEmpty(splitKeys)) { + // documents size is less than chunk size, treat the entire collection as single chunk. + return SingleSplitStrategy.INSTANCE.split(splitContext); + } + + SeaTunnelRowType rowType = shardKeysToRowType(Collections.singleton(ID_FIELD)); + List snapshotSplits = new ArrayList<>(splitKeys.size() + 1); + + BsonValue lowerValue = new BsonMinKey(); + ; + for (int i = 0; i < splitKeys.size(); i++) { + BsonValue splitKeyValue = splitKeys.get(i).asDocument().get(ID_FIELD); + snapshotSplits.add( + new SnapshotSplit( + splitId(collectionId, i), + collectionId, + rowType, + boundOfId(lowerValue), + boundOfId(splitKeyValue), + null)); + lowerValue = splitKeyValue; + } + + SnapshotSplit lastSplit = + new SnapshotSplit( + splitId(collectionId, splitKeys.size()), + collectionId, + rowType, + boundOfId(lowerValue), + maxUpperBoundOfId(), + null); + snapshotSplits.add(lastSplit); + + return snapshotSplits; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/BsonUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/BsonUtils.java new file mode 100644 index 000000000000..badd68305f27 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/BsonUtils.java @@ -0,0 +1,324 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import org.bson.BsonArray; +import org.bson.BsonBinary; +import org.bson.BsonDbPointer; +import org.bson.BsonDocument; +import org.bson.BsonJavaScriptWithScope; +import org.bson.BsonNumber; +import org.bson.BsonObjectId; +import org.bson.BsonString; +import org.bson.BsonType; +import org.bson.BsonUndefined; +import org.bson.BsonValue; +import org.bson.types.Decimal128; +import org.jetbrains.annotations.NotNull; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.Objects; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; + +public class BsonUtils { + + public static int compareBsonValue(BsonValue o1, BsonValue o2) { + return compareBsonValue(o1, o2, true); + } + + private static int compareBsonValue(BsonValue o1, BsonValue o2, boolean isTopLevel) { + if (Objects.equals(o1, o2)) { + return 0; + } + + if (isTopLevel) { + BsonValue element1 = o1; + BsonValue element2 = o2; + + if (o1 != null && o1.isArray()) { + element1 = smallestValueOfArray(o1.asArray()); + } + if (o2.isArray()) { + element2 = smallestValueOfArray(o2.asArray()); + } + return compareBsonValues(element1, element2); + } + if (typeOrder(o1) != typeOrder(o2)) { + return Integer.compare(typeOrder(o1), typeOrder(o2)); + } + + if (isNull(o1) || isMinKey(o1) || isMaxKey(o1)) { + return 0; // Null == Null, MinKey == MinKey, MaxKey == MaxKey + } + + switch (o1.getBsonType()) { + case INT32: + case INT64: + case DOUBLE: + return compareBsonNumbers(o1.asNumber(), o2.asNumber()); + case STRING: + case JAVASCRIPT: + case REGULAR_EXPRESSION: + return compareStrings(o1.asString().getValue(), o2.asString().getValue()); + case BOOLEAN: + return compareBooleans(o1.asBoolean().getValue(), o2.asBoolean().getValue()); + case DATE_TIME: + return compareDateTimes(o1.asDateTime().getValue(), o2.asDateTime().getValue()); + case TIMESTAMP: + return compareTimestamps(o1.asTimestamp().getValue(), o2.asTimestamp().getValue()); + case BINARY: + return compareBsonBinary(o1.asBinary(), o2.asBinary()); + case OBJECT_ID: + return o1.asObjectId().compareTo(o2.asObjectId()); + case DOCUMENT: + case DB_POINTER: + return compareBsonDocument(toBsonDocument(o1), toBsonDocument(o2)); + case ARRAY: + return compareBsonArray(o1.asArray(), o2.asArray()); + case JAVASCRIPT_WITH_SCOPE: + return compareJavascriptWithScope( + o1.asJavaScriptWithScope(), o2.asJavaScriptWithScope()); + default: + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + String.format("Unable to compare bson values between %s and %s", o1, o2)); + } + } + + private static int compareBsonValues(BsonValue v1, BsonValue v2) { + return compareBsonValue(v1, v2, false); + } + + private static int compareBsonNumbers(BsonNumber n1, BsonNumber n2) { + Decimal128 decimal1 = getDecimal128FromCache(n1); + Decimal128 decimal2 = getDecimal128FromCache(n2); + return decimal1.compareTo(decimal2); + } + + private static int compareStrings(String s1, String s2) { + return getStringFromCache(s1).compareTo(getStringFromCache(s2)); + } + + private static int compareBooleans(boolean b1, boolean b2) { + return Boolean.compare(b1, b2); + } + + private static int compareDateTimes(long dt1, long dt2) { + return Long.compare(dt1, dt2); + } + + private static int compareTimestamps(long ts1, long ts2) { + return Long.compare(ts1, ts2); + } + + private static final Map decimalCache = new HashMap<>(); + private static final Map stringCache = new HashMap<>(); + + private static Decimal128 getDecimal128FromCache(BsonValue value) { + return decimalCache.computeIfAbsent(value, BsonUtils::toDecimal128); + } + + private static String getStringFromCache(String value) { + return stringCache.computeIfAbsent(value, k -> k); + } + + public static int compareBsonDocument(@NotNull BsonDocument d1, @NotNull BsonDocument d2) { + Iterator> iterator1 = d1.entrySet().iterator(); + Iterator> iterator2 = d2.entrySet().iterator(); + + if (!iterator1.hasNext() && !iterator2.hasNext()) { + return 0; + } else if (!iterator1.hasNext()) { + return -1; + } else if (!iterator2.hasNext()) { + return 1; + } else { + while (iterator1.hasNext() && iterator2.hasNext()) { + Map.Entry entry1 = iterator1.next(); + Map.Entry entry2 = iterator2.next(); + + int result = + Integer.compare(typeOrder(entry1.getValue()), typeOrder(entry2.getValue())); + if (result != 0) { + return result; + } + + result = entry1.getKey().compareTo(entry2.getKey()); + if (result != 0) { + return result; + } + + result = compareBsonValue(entry1.getValue(), entry2.getValue(), false); + if (result != 0) { + return result; + } + } + + return Integer.compare(d1.size(), d2.size()); + } + } + + public static int compareBsonArray(BsonArray a1, BsonArray a2) { + return compareBsonValue(smallestValueOfArray(a1), smallestValueOfArray(a2), false); + } + + private static BsonValue smallestValueOfArray(@NotNull BsonArray bsonArray) { + if (bsonArray.isEmpty()) { + return new BsonUndefined(); + } + + if (bsonArray.size() == 1) { + return bsonArray.get(0); + } + + return bsonArray.getValues().stream() + .min((e1, e2) -> compareBsonValue(e1, e2, false)) + .orElseThrow( + () -> + new IllegalStateException( + "Unable to find smallest value in the array.")); + } + + public static int compareBsonBinary(@NotNull BsonBinary b1, @NotNull BsonBinary b2) { + byte[] data1 = b1.getData(); + byte[] data2 = b2.getData(); + + int lengthComparison = Integer.compare(data1.length, data2.length); + if (lengthComparison != 0) { + return lengthComparison; + } + + int typeComparison = Byte.compare(b1.getType(), b2.getType()); + if (typeComparison != 0) { + return typeComparison; + } + + for (int i = 0; i < data1.length; i++) { + int byteComparison = Integer.compareUnsigned(data1[i] & 0xff, data2[i] & 0xff); + if (byteComparison != 0) { + return byteComparison; + } + } + + return 0; + } + + public static int compareJavascriptWithScope( + @NotNull BsonJavaScriptWithScope c1, @NotNull BsonJavaScriptWithScope c2) { + int result = c1.getCode().compareTo(c2.getCode()); + if (result != 0) { + return result; + } + return compareBsonDocument(c1.getScope(), c2.getScope()); + } + + public static boolean isNull(BsonValue bsonValue) { + return bsonValue == null + || bsonValue.isNull() + || bsonValue.getBsonType() == BsonType.UNDEFINED; + } + + public static boolean isMinKey(BsonValue bsonValue) { + return bsonValue != null && bsonValue.getBsonType() == BsonType.MIN_KEY; + } + + public static boolean isMaxKey(BsonValue bsonValue) { + return bsonValue != null && bsonValue.getBsonType() == BsonType.MAX_KEY; + } + + public static Decimal128 toDecimal128(@NotNull BsonValue bsonValue) { + if (bsonValue.isNumber()) { + return bsonValue.asNumber().decimal128Value(); + } else if (bsonValue.isDecimal128()) { + return bsonValue.asDecimal128().decimal128Value(); + } else { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Cannot convert to Decimal128 with unexpected value: " + bsonValue); + } + } + + public static BsonDocument toBsonDocument(@NotNull BsonValue bsonValue) { + if (bsonValue.isDocument()) { + return bsonValue.asDocument(); + } else if (bsonValue.isDBPointer()) { + BsonDbPointer dbPointer = bsonValue.asDBPointer(); + return new BsonDocument("$ref", new BsonString(dbPointer.getNamespace())) + .append("$id", new BsonObjectId(dbPointer.getId())); + } + + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Cannot convert to Document with unexpected value: " + bsonValue); + } + + public static int typeOrder(BsonValue bsonValue) { + // Missing Key field + if (bsonValue == null) { + return 3; + } + + BsonType bsonType = bsonValue.getBsonType(); + switch (bsonType) { + case MIN_KEY: + return 1; + case UNDEFINED: + return 2; + case NULL: + return 3; + case INT32: + case INT64: + case DOUBLE: + case DECIMAL128: + return 4; + case STRING: + case SYMBOL: + return 5; + case DOCUMENT: + case DB_POINTER: + return 6; + case ARRAY: + return 7; + case BINARY: + return 8; + case OBJECT_ID: + return 9; + case BOOLEAN: + return 10; + case DATE_TIME: + return 11; + case TIMESTAMP: + return 12; + case REGULAR_EXPRESSION: + return 13; + case JAVASCRIPT: + return 14; + case JAVASCRIPT_WITH_SCOPE: + return 15; + case MAX_KEY: + return 99; + default: + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Unknown bson type : " + bsonType); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ChunkUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ChunkUtils.java new file mode 100644 index 000000000000..f7180cdd3550 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ChunkUtils.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonMaxKey; +import org.bson.BsonMinKey; +import org.bson.BsonValue; +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; + +public class ChunkUtils { + + private ChunkUtils() {} + + @Contract("_ -> new") + public static Object @NotNull [] boundOfId(BsonValue bound) { + return new Object[] { + new BsonDocument(ID_FIELD, new BsonInt32(1)), new BsonDocument(ID_FIELD, bound) + }; + } + + @Contract(" -> new") + public static Object @NotNull [] minLowerBoundOfId() { + return boundOfId(new BsonMinKey()); + } + + @Contract(" -> new") + public static Object @NotNull [] maxUpperBoundOfId() { + return boundOfId(new BsonMaxKey()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/CollectionDiscoveryUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/CollectionDiscoveryUtils.java new file mode 100644 index 000000000000..868a3ed73e03 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/CollectionDiscoveryUtils.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.commons.collections4.CollectionUtils; + +import org.bson.BsonDocument; +import org.bson.conversions.Bson; +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.MongoNamespace; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoDatabase; +import lombok.AllArgsConstructor; +import lombok.Getter; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ADD_NS_FIELD_NAME; + +public class CollectionDiscoveryUtils { + + public static final Bson ADD_NS_FIELD = + BsonDocument.parse( + String.format( + "{'$addFields': {'%s': {'$concat': ['$ns.db', '.', '$ns.coll']}}}", + ADD_NS_FIELD_NAME)); + + private CollectionDiscoveryUtils() {} + + public static @NotNull List databaseNames( + @NotNull MongoClient mongoClient, Predicate databaseFilter) { + List databaseNames = new ArrayList<>(); + return mongoClient.listDatabaseNames().into(databaseNames).stream() + .filter(databaseFilter) + .collect(Collectors.toList()); + } + + public static @NotNull List collectionNames( + MongoClient mongoClient, + List databaseNames, + Predicate collectionFilter) { + return collectionNames(mongoClient, databaseNames, collectionFilter, String::toString); + } + + public static @NotNull List collectionNames( + MongoClient mongoClient, + @NotNull List databaseNames, + Predicate collectionFilter, + Function conversion) { + List collectionNames = new ArrayList<>(); + for (String dbName : databaseNames) { + MongoDatabase db = mongoClient.getDatabase(dbName); + StreamSupport.stream(db.listCollectionNames().spliterator(), false) + .map(collName -> dbName + "." + collName) + .filter(collectionFilter) + .map(conversion) + .forEach(collectionNames::add); + } + return collectionNames; + } + + private static Predicate stringListFilter( + Predicate filter, List stringList) { + if (CollectionUtils.isNotEmpty(stringList)) { + List databasePatterns = includeListAsPatterns(stringList); + filter = filter.and(anyMatch(databasePatterns)); + } + return filter; + } + + public static Predicate databaseFilter(List databaseList) { + return stringListFilter(CollectionDiscoveryUtils::isNotBuiltInDatabase, databaseList); + } + + public static Predicate collectionsFilter(List collectionList) { + return stringListFilter(CollectionDiscoveryUtils::isNotBuiltInCollections, collectionList); + } + + @Contract(pure = true) + public static @NotNull Predicate anyMatch(List patterns) { + return s -> patterns.stream().anyMatch(p -> p.matcher(s).matches()); + } + + public static Pattern includeListAsFlatPattern(List includeList) { + return includeListAsFlatPattern(includeList, CollectionDiscoveryUtils::completionPattern); + } + + public static Pattern includeListAsFlatPattern( + List includeList, Function conversion) { + if (includeList == null || includeList.isEmpty()) { + return null; + } + String flatPatternLiteral = + includeList.stream() + .map(conversion) + .map(Pattern::pattern) + .collect(Collectors.joining("|")); + + return Pattern.compile(flatPatternLiteral); + } + + public static List includeListAsPatterns(List includeList) { + return includeListAsPatterns(includeList, CollectionDiscoveryUtils::completionPattern); + } + + public static List includeListAsPatterns( + List includeList, Function conversion) { + return includeList != null && !includeList.isEmpty() + ? includeList.stream().map(conversion).collect(Collectors.toList()) + : Collections.emptyList(); + } + + public static boolean isNotBuiltInCollections(String fullName) { + if (fullName == null) { + return false; + } + MongoNamespace namespace = new MongoNamespace(fullName); + return isNotBuiltInDatabase(namespace.getDatabaseName()) + && !namespace.getCollectionName().startsWith("system."); + } + + public static boolean isNotBuiltInDatabase(String databaseName) { + if (databaseName == null) { + return false; + } + return !"local".equals(databaseName) + && !"admin".equals(databaseName) + && !"config".equals(databaseName); + } + + public static @NotNull Pattern completionPattern(@NotNull String pattern) { + if (pattern.startsWith("^") && pattern.endsWith("$")) { + return Pattern.compile(pattern); + } + return Pattern.compile("^(" + pattern + ")$"); + } + + @Getter + @AllArgsConstructor + public static class CollectionDiscoveryInfo implements Serializable { + private static final long serialVersionUID = 1L; + + private final List discoveredDatabases; + + private final List discoveredCollections; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java new file mode 100644 index 000000000000..e1caa345b9fc --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbRecordUtils.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.commons.lang3.StringUtils; +import org.apache.kafka.connect.data.SchemaAndValue; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; + +import org.bson.BsonDocument; +import org.bson.BsonTimestamp; +import org.bson.json.JsonWriterSettings; +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Unmodifiable; + +import com.mongodb.kafka.connect.source.json.formatter.DefaultJson; +import com.mongodb.kafka.connect.source.schema.AvroSchemaDefaults; +import com.mongodb.kafka.connect.source.schema.BsonValueToSchemaAndValue; +import io.debezium.relational.TableId; + +import java.time.Instant; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static com.mongodb.kafka.connect.source.schema.AvroSchema.fromJson; +import static org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkEvent.isWatermarkEvent; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.COLL_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DB_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.OUTPUT_SCHEMA; + +public class MongodbRecordUtils { + + public static boolean isHeartbeatEvent(SourceRecord sourceRecord) { + return "true".equals(getOffsetValue(sourceRecord, "copy")); + } + + public static boolean isDataChangeRecord(SourceRecord sourceRecord) { + return !isWatermarkEvent(sourceRecord) && !isHeartbeatEvent(sourceRecord); + } + + public static BsonDocument getResumeToken(SourceRecord sourceRecord) { + return BsonDocument.parse(getOffsetValue(sourceRecord, ID_FIELD)); + } + + public static BsonDocument getDocumentKey(@NotNull SourceRecord sourceRecord) { + Struct value = (Struct) sourceRecord.value(); + return BsonDocument.parse(value.getString(DOCUMENT_KEY)); + } + + public static String getOffsetValue(@NotNull SourceRecord sourceRecord, String key) { + return (String) sourceRecord.sourceOffset().get(key); + } + + public static @NotNull TableId getTableId(@NotNull SourceRecord dataRecord) { + Struct value = (Struct) dataRecord.value(); + Struct source = value.getStruct(NS_FIELD); + String dbName = source.getString(DB_FIELD); + String collName = source.getString(COLL_FIELD); + return new TableId(dbName, null, collName); + } + + @Contract(" -> new") + public static @NotNull BsonTimestamp currentBsonTimestamp() { + return bsonTimestampFromEpochMillis(System.currentTimeMillis()); + } + + @Contract(value = " -> new", pure = true) + public static @NotNull BsonTimestamp maximumBsonTimestamp() { + return new BsonTimestamp(Integer.MAX_VALUE, Integer.MAX_VALUE); + } + + @Contract("_ -> new") + public static @NotNull BsonTimestamp bsonTimestampFromEpochMillis(long epochMillis) { + return new BsonTimestamp((int) Instant.ofEpochMilli(epochMillis).getEpochSecond(), 1); + } + + public static @NotNull SourceRecord buildSourceRecord( + final Map partition, + final Map sourceOffset, + final String topicName, + final BsonDocument keyDocument, + final BsonDocument valueDocument) { + return buildSourceRecord( + partition, + sourceOffset, + topicName, + keyDocument, + valueDocument, + new DefaultJson().getJsonWriterSettings()); + } + + public static @NotNull SourceRecord buildSourceRecord( + Map partition, + Map sourceOffset, + String topicName, + BsonDocument keyDocument, + BsonDocument valueDocument, + JsonWriterSettings jsonWriterSettings) { + BsonValueToSchemaAndValue schemaAndValue = + new BsonValueToSchemaAndValue(jsonWriterSettings); + SchemaAndValue keySchemaAndValue = + schemaAndValue.toSchemaAndValue( + fromJson(AvroSchemaDefaults.DEFAULT_AVRO_KEY_SCHEMA), keyDocument); + SchemaAndValue valueSchemaAndValue = + schemaAndValue.toSchemaAndValue(fromJson(OUTPUT_SCHEMA), valueDocument); + + return new SourceRecord( + partition, + sourceOffset, + topicName, + keySchemaAndValue.schema(), + keySchemaAndValue.value(), + valueSchemaAndValue.schema(), + valueSchemaAndValue.value()); + } + + public static @NotNull Map createSourceOffsetMap( + @NotNull BsonDocument idDocument, boolean isSnapshotRecord) { + Map sourceOffset = new HashMap<>(); + sourceOffset.put(ID_FIELD, idDocument.toJson()); + sourceOffset.put("copy", String.valueOf(isSnapshotRecord)); + return sourceOffset; + } + + public static @NotNull @Unmodifiable Map createPartitionMap( + String hosts, String database, String collection) { + StringBuilder builder = new StringBuilder(); + builder.append("mongodb://"); + builder.append(hosts); + builder.append("/"); + if (StringUtils.isNotEmpty(database)) { + builder.append(database); + } + if (StringUtils.isNotEmpty(collection)) { + builder.append("."); + builder.append(collection); + } + return Collections.singletonMap(NS_FIELD, builder.toString()); + } + + @Contract(pure = true) + public static @NotNull @Unmodifiable Map createHeartbeatPartitionMap( + String hosts) { + String builder = "mongodb://" + hosts + "/" + "__mongodb_heartbeats"; + return Collections.singletonMap(NS_FIELD, builder); + } + + @Contract(value = "_ -> new", pure = true) + public static @NotNull @Unmodifiable Map createWatermarkPartitionMap( + String partition) { + return Collections.singletonMap(NS_FIELD, partition); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbUtils.java new file mode 100644 index 000000000000..bb42dbfa1973 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/MongodbUtils.java @@ -0,0 +1,408 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.internal.MongodbClientProvider; +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.source.offset.ChangeStreamDescriptor; + +import org.apache.commons.lang3.StringUtils; + +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonString; +import org.bson.BsonTimestamp; +import org.bson.Document; +import org.bson.conversions.Bson; +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; + +import com.mongodb.ConnectionString; +import com.mongodb.client.ChangeStreamIterable; +import com.mongodb.client.MongoChangeStreamCursor; +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.changestream.ChangeStreamDocument; +import com.mongodb.client.model.changestream.FullDocument; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.regex.Pattern; + +import static com.mongodb.client.model.Aggregates.match; +import static com.mongodb.client.model.Filters.and; +import static com.mongodb.client.model.Filters.eq; +import static com.mongodb.client.model.Filters.or; +import static com.mongodb.client.model.Filters.regex; +import static com.mongodb.client.model.Projections.include; +import static com.mongodb.client.model.Sorts.ascending; +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ADD_NS_FIELD_NAME; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.COMMAND_SUCCEED_FLAG; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DOCUMENT_KEY; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.DROPPED_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.ID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MAX_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.MIN_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.SHARD_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.config.MongodbSourceOptions.UUID_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.ADD_NS_FIELD; +import static org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils.CollectionDiscoveryUtils.includeListAsFlatPattern; + +@Slf4j +public class MongodbUtils { + + private static final Map> cache = new ConcurrentHashMap<>(); + + public static ChangeStreamDescriptor getChangeStreamDescriptor( + @NotNull MongodbSourceConfig sourceConfig, + List discoveredDatabases, + List discoveredCollections) { + List databaseList = sourceConfig.getDatabaseList(); + List collectionList = sourceConfig.getCollectionList(); + + ChangeStreamDescriptor changeStreamFilter; + if (collectionList != null) { + // Watching collections changes + if (isIncludeListExplicitlySpecified(collectionList, discoveredCollections)) { + changeStreamFilter = + ChangeStreamDescriptor.collection( + TableId.parse(discoveredCollections.get(0))); + } else { + Pattern namespaceRegex = includeListAsFlatPattern(collectionList); + if (databaseList != null) { + if (isIncludeListExplicitlySpecified(databaseList, discoveredDatabases)) { + changeStreamFilter = + ChangeStreamDescriptor.database( + discoveredDatabases.get(0), namespaceRegex); + } else { + Pattern databaseRegex = includeListAsFlatPattern(databaseList); + changeStreamFilter = + ChangeStreamDescriptor.deployment(databaseRegex, namespaceRegex); + } + } else { + changeStreamFilter = ChangeStreamDescriptor.deployment(null, namespaceRegex); + } + } + } else if (databaseList != null) { + if (isIncludeListExplicitlySpecified(databaseList, discoveredDatabases)) { + changeStreamFilter = ChangeStreamDescriptor.database(discoveredDatabases.get(0)); + } else { + Pattern databaseRegex = includeListAsFlatPattern(databaseList); + changeStreamFilter = ChangeStreamDescriptor.deployment(databaseRegex); + } + } else { + // Watching all changes on the cluster + changeStreamFilter = ChangeStreamDescriptor.deployment(); + } + return changeStreamFilter; + } + + public static boolean isIncludeListExplicitlySpecified( + List includeList, List discoveredList) { + if (includeList == null || includeList.size() != 1) { + return false; + } + if (discoveredList == null || discoveredList.size() != 1) { + return false; + } + String firstOfIncludeList = includeList.get(0); + String firstOfDiscoveredList = discoveredList.get(0); + return firstOfDiscoveredList.equals(firstOfIncludeList); + } + + public static @NotNull ChangeStreamIterable getChangeStreamIterable( + MongodbSourceConfig sourceConfig, @NotNull ChangeStreamDescriptor descriptor) { + return getChangeStreamIterable( + createMongoClient(sourceConfig), + descriptor.getDatabase(), + descriptor.getCollection(), + descriptor.getDatabaseRegex(), + descriptor.getNamespaceRegex(), + sourceConfig.getBatchSize(), + sourceConfig.isUpdateLookup()); + } + + public static @NotNull ChangeStreamIterable getChangeStreamIterable( + MongoClient mongoClient, + @NotNull ChangeStreamDescriptor descriptor, + int batchSize, + boolean updateLookup) { + return getChangeStreamIterable( + mongoClient, + descriptor.getDatabase(), + descriptor.getCollection(), + descriptor.getDatabaseRegex(), + descriptor.getNamespaceRegex(), + batchSize, + updateLookup); + } + + public static @NotNull ChangeStreamIterable getChangeStreamIterable( + MongoClient mongoClient, + String database, + String collection, + Pattern databaseRegex, + Pattern namespaceRegex, + int batchSize, + boolean updateLookup) { + ChangeStreamIterable changeStream; + if (StringUtils.isNotEmpty(database) && StringUtils.isNotEmpty(collection)) { + MongoCollection coll = + mongoClient.getDatabase(database).getCollection(collection); + log.info("Preparing change stream for collection {}.{}", database, collection); + changeStream = coll.watch(); + } else if (StringUtils.isNotEmpty(database) && namespaceRegex != null) { + MongoDatabase db = mongoClient.getDatabase(database); + List pipeline = new ArrayList<>(); + pipeline.add(ADD_NS_FIELD); + Bson nsFilter = regex(ADD_NS_FIELD_NAME, namespaceRegex); + pipeline.add(match(nsFilter)); + log.info( + "Preparing change stream for database {} with namespace regex filter {}", + database, + namespaceRegex); + changeStream = db.watch(pipeline); + } else if (StringUtils.isNotEmpty(database)) { + MongoDatabase db = mongoClient.getDatabase(database); + log.info("Preparing change stream for database {}", database); + changeStream = db.watch(); + } else if (namespaceRegex != null) { + List pipeline = new ArrayList<>(); + pipeline.add(ADD_NS_FIELD); + + Bson nsFilter = regex(ADD_NS_FIELD_NAME, namespaceRegex); + if (databaseRegex != null) { + Bson dbFilter = regex("ns.db", databaseRegex); + nsFilter = and(dbFilter, nsFilter); + log.info( + "Preparing change stream for deployment with" + + " database regex filter {} and namespace regex filter {}", + databaseRegex, + namespaceRegex); + } else { + log.info( + "Preparing change stream for deployment with namespace regex filter {}", + namespaceRegex); + } + + pipeline.add(match(nsFilter)); + changeStream = mongoClient.watch(pipeline); + } else if (databaseRegex != null) { + List pipeline = new ArrayList<>(); + pipeline.add(match(regex("ns.db", databaseRegex))); + + log.info( + "Preparing change stream for deployment with database regex filter {}", + databaseRegex); + changeStream = mongoClient.watch(pipeline); + } else { + log.info("Preparing change stream for deployment"); + changeStream = mongoClient.watch(); + } + + if (batchSize > 0) { + changeStream.batchSize(batchSize); + } + + if (updateLookup) { + changeStream.fullDocument(FullDocument.UPDATE_LOOKUP); + } + return changeStream; + } + + public static BsonDocument getLatestResumeToken( + MongoClient mongoClient, ChangeStreamDescriptor descriptor) { + ChangeStreamIterable changeStreamIterable = + getChangeStreamIterable(mongoClient, descriptor, 1, false); + + // Nullable when no change record or postResumeToken (new in MongoDB 4.0.7). + try (MongoChangeStreamCursor> changeStreamCursor = + changeStreamIterable.cursor()) { + ChangeStreamDocument firstResult = changeStreamCursor.tryNext(); + + return firstResult != null + ? firstResult.getResumeToken() + : changeStreamCursor.getResumeToken(); + } + } + + public static boolean isCommandSucceed(BsonDocument commandResult) { + return commandResult != null && COMMAND_SUCCEED_FLAG.equals(commandResult.getDouble("ok")); + } + + public static String commandErrorMessage(BsonDocument commandResult) { + return Optional.ofNullable(commandResult) + .map(doc -> doc.getString("errmsg")) + .map(BsonString::getValue) + .orElse(null); + } + + public static @NotNull BsonDocument collStats( + @NotNull MongoClient mongoClient, @NotNull TableId collectionId) { + BsonDocument collStatsCommand = + new BsonDocument("collStats", new BsonString(collectionId.table())); + return mongoClient + .getDatabase(collectionId.catalog()) + .runCommand(collStatsCommand, BsonDocument.class); + } + + public static @NotNull BsonDocument splitVector( + MongoClient mongoClient, + TableId collectionId, + BsonDocument keyPattern, + int maxChunkSizeMB) { + return splitVector(mongoClient, collectionId, keyPattern, maxChunkSizeMB, null, null); + } + + public static @NotNull BsonDocument splitVector( + @NotNull MongoClient mongoClient, + @NotNull TableId collectionId, + BsonDocument keyPattern, + int maxChunkSizeMB, + BsonDocument min, + BsonDocument max) { + BsonDocument splitVectorCommand = + new BsonDocument("splitVector", new BsonString(collectionId.identifier())) + .append("keyPattern", keyPattern) + .append("maxChunkSize", new BsonInt32(maxChunkSizeMB)); + Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v)); + Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v)); + return mongoClient + .getDatabase(collectionId.catalog()) + .runCommand(splitVectorCommand, BsonDocument.class); + } + + public static BsonTimestamp getCurrentClusterTime(MongoClient mongoClient) { + BsonDocument isMasterResult = isMaster(mongoClient); + if (!isCommandSucceed(isMasterResult)) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, + "Failed to execute isMaster command: " + commandErrorMessage(isMasterResult)); + } + return isMasterResult.getDocument("$clusterTime").getTimestamp("clusterTime"); + } + + public static @NotNull BsonDocument isMaster(@NotNull MongoClient mongoClient) { + BsonDocument isMasterCommand = new BsonDocument("isMaster", new BsonInt32(1)); + return mongoClient.getDatabase("admin").runCommand(isMasterCommand, BsonDocument.class); + } + + public static @NotNull List readChunks( + MongoClient mongoClient, @NotNull BsonDocument collectionMetadata) { + MongoCollection chunks = + getMongoCollection(mongoClient, TableId.parse("config.chunks"), BsonDocument.class); + List collectionChunks = new ArrayList<>(); + + Bson filter = + or( + new BsonDocument(NS_FIELD, collectionMetadata.get(ID_FIELD)), + // MongoDB 4.9.0 removed ns field of config.chunks collection, using + // collection's uuid instead. + // See: https://jira.mongodb.org/browse/SERVER-53105 + new BsonDocument(UUID_FIELD, collectionMetadata.get(UUID_FIELD))); + + chunks.find(filter) + .projection(include(MIN_FIELD, MAX_FIELD, SHARD_FIELD)) + .sort(ascending(MIN_FIELD)) + .into(collectionChunks); + return collectionChunks; + } + + public static BsonDocument readCollectionMetadata( + MongoClient mongoClient, @NotNull TableId collectionId) { + MongoCollection collection = + getMongoCollection( + mongoClient, TableId.parse("config.collections"), BsonDocument.class); + + return collection + .find(eq(ID_FIELD, collectionId.identifier())) + .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, DOCUMENT_KEY)) + .first(); + } + + public static @NotNull MongoCollection getMongoCollection( + MongoClient mongoClient, TableId collectionId, Class documentClass) { + return getCollection(mongoClient, collectionId, documentClass); + } + + @SuppressWarnings("unchecked") + public static @NotNull MongoCollection getCollection( + MongoClient mongoClient, TableId collectionId, Class documentClass) { + MongoCollection cachedCollection = cache.get(collectionId); + if (cachedCollection == null) { + MongoCollection collection = + mongoClient + .getDatabase(collectionId.catalog()) + .getCollection(collectionId.table(), documentClass); + cache.put(collectionId, collection); + return collection; + } + return (MongoCollection) cachedCollection; + } + + public static MongoClient createMongoClient(MongodbSourceConfig sourceConfig) { + return MongodbClientProvider.INSTANCE.getOrCreateMongoClient(sourceConfig); + } + + @Contract("_, _, _, _ -> new") + public static @NotNull ConnectionString buildConnectionString( + String username, String password, String hosts, String connectionOptions) { + StringBuilder sb = new StringBuilder("mongodb://"); + + if (hasCredentials(username, password)) { + appendCredentials(sb, username, password); + } + + sb.append(hosts); + + if (StringUtils.isNotEmpty(connectionOptions)) { + sb.append("/?").append(connectionOptions); + } + + return new ConnectionString(sb.toString()); + } + + private static boolean hasCredentials(String username, String password) { + return StringUtils.isNotEmpty(username) && StringUtils.isNotEmpty(password); + } + + private static void appendCredentials( + @NotNull StringBuilder sb, String username, String password) { + sb.append(encodeValue(username)).append(":").append(encodeValue(password)).append("@"); + } + + public static String encodeValue(String value) { + try { + return URLEncoder.encode(value, StandardCharsets.UTF_8.name()); + } catch (UnsupportedEncodingException e) { + throw new MongodbConnectorException(ILLEGAL_ARGUMENT, e.getMessage()); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java new file mode 100644 index 000000000000..8fef921808cb --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/mongodb/utils/ResumeToken.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.utils; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.exception.MongodbConnectorException; + +import org.bson.BsonDocument; +import org.bson.BsonTimestamp; +import org.bson.BsonValue; +import org.jetbrains.annotations.Contract; +import org.jetbrains.annotations.NotNull; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Objects; + +import static org.apache.seatunnel.common.exception.CommonErrorCode.ILLEGAL_ARGUMENT; + +public class ResumeToken { + + private static final int K_TIMESTAMP = 130; + + @Contract("_ -> new") + public static @NotNull BsonTimestamp decodeTimestamp(BsonDocument resumeToken) { + Objects.requireNonNull(resumeToken, "Missing ResumeToken."); + BsonValue bsonValue = resumeToken.get("_data"); + byte[] keyStringBytes = extractKeyStringBytes(bsonValue); + validateKeyType(keyStringBytes); + + ByteBuffer buffer = ByteBuffer.wrap(keyStringBytes).order(ByteOrder.BIG_ENDIAN); + int t = buffer.getInt(); + int i = buffer.getInt(); + return new BsonTimestamp(t, i); + } + + private static byte[] extractKeyStringBytes(@NotNull BsonValue bsonValue) { + if (bsonValue.isBinary()) { + return bsonValue.asBinary().getData(); + } else if (bsonValue.isString()) { + return hexToUint8Array(bsonValue.asString().getValue()); + } else { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Unknown resume token format: " + bsonValue); + } + } + + @Contract(pure = true) + private static void validateKeyType(byte @NotNull [] keyStringBytes) { + int kType = keyStringBytes[0] & 0xff; + if (kType != K_TIMESTAMP) { + throw new MongodbConnectorException( + ILLEGAL_ARGUMENT, "Unknown keyType of timestamp: " + kType); + } + } + + private static byte @NotNull [] hexToUint8Array(@NotNull String str) { + int len = str.length(); + byte[] data = new byte[len / 2]; + for (int i = 0; i < len; i += 2) { + data[i / 2] = + (byte) + ((Character.digit(str.charAt(i), 16) << 4) + + Character.digit(str.charAt(i + 1), 16)); + } + return data; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/test/java/mongodb/source/MySqlIncrementalSourceFactoryTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/test/java/mongodb/source/MySqlIncrementalSourceFactoryTest.java new file mode 100644 index 000000000000..09093d6991e5 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-mongodb/src/test/java/mongodb/source/MySqlIncrementalSourceFactoryTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package mongodb.source; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mongodb.MongodbIncrementalSourceFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class MySqlIncrementalSourceFactoryTest { + @Test + public void testOptionRule() { + Assertions.assertNotNull((new MongodbIncrementalSourceFactory()).optionRule()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/pom.xml b/seatunnel-connectors-v2/connector-cdc/pom.xml index 492dd7143626..38f52b1680d7 100644 --- a/seatunnel-connectors-v2/connector-cdc/pom.xml +++ b/seatunnel-connectors-v2/connector-cdc/pom.xml @@ -33,6 +33,7 @@ connector-cdc-base connector-cdc-mysql connector-cdc-sqlserver + connector-cdc-mongodb diff --git a/seatunnel-dist/pom.xml b/seatunnel-dist/pom.xml index 51d4f4c48a99..504aef06578c 100644 --- a/seatunnel-dist/pom.xml +++ b/seatunnel-dist/pom.xml @@ -464,6 +464,12 @@ ${project.version} provided + + org.apache.seatunnel + connector-cdc-mongodb + ${project.version} + provided + org.apache.seatunnel connector-cdc-sqlserver diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/pom.xml new file mode 100644 index 000000000000..38eda0a913d6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/pom.xml @@ -0,0 +1,60 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-cdc-mongodb-e2e + SeaTunnel : E2E : Connector V2 : CDC Mongodb + + + 8 + 8 + UTF-8 + + + + + org.apache.seatunnel + connector-cdc-mongodb + ${project.version} + test + + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test + + + org.apache.seatunnel + connector-cdc-mysql + ${project.version} + test-jar + test + + + org.testcontainers + mysql + ${testcontainer.version} + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java new file mode 100644 index 000000000000..c33f6d047d48 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongoDBContainer.java @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package mongodb; + +import org.apache.commons.lang3.StringUtils; + +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.Network; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.containers.wait.strategy.WaitStrategy; +import org.testcontainers.images.builder.ImageFromDockerfile; + +import com.github.dockerjava.api.command.InspectContainerResponse; +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertNotNull; + +@Slf4j +public class MongoDBContainer extends GenericContainer { + + private static final String DOCKER_IMAGE_NAME = "mongo:5.0.2"; + + public static final int MONGODB_PORT = 27017; + + public static final String MONGO_SUPER_USER = "superuser"; + + public static final String MONGO_SUPER_PASSWORD = "superpw"; + + private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)//.*$"); + + private final ShardingClusterRole clusterRole; + + public MongoDBContainer(Network network) { + this(network, ShardingClusterRole.NONE); + } + + public MongoDBContainer(Network network, ShardingClusterRole clusterRole) { + super( + new ImageFromDockerfile() + .withFileFromClasspath("random.key", "docker/mongodb/random.key") + .withFileFromClasspath("setup.js", "docker/mongodb/setup.js") + .withDockerfileFromBuilder( + builder -> + builder.from(DOCKER_IMAGE_NAME) + .copy( + "setup.js", + "/docker-entrypoint-initdb.d/setup.js") + .copy("random.key", "/data/keyfile/random.key") + .run("chown mongodb /data/keyfile/random.key") + .run("chmod 400 /data/keyfile/random.key") + .env("MONGO_INITDB_ROOT_USERNAME", MONGO_SUPER_USER) + .env( + "MONGO_INITDB_ROOT_PASSWORD", + MONGO_SUPER_PASSWORD) + .env("MONGO_INITDB_DATABASE", "admin") + .build())); + this.clusterRole = clusterRole; + + withNetwork(network); + withNetworkAliases(clusterRole.hostname); + withExposedPorts(MONGODB_PORT); + withCommand(ShardingClusterRole.startupCommand(clusterRole)); + waitingFor(clusterRole.waitStrategy); + } + + public void executeCommand(String command) { + try { + log.info("Executing mongo command: {}", command); + ExecResult execResult = + execInContainer( + "mongo", + "-u", + MONGO_SUPER_USER, + "-p", + MONGO_SUPER_PASSWORD, + "--eval", + command); + log.info(execResult.getStdout()); + if (execResult.getExitCode() != 0) { + throw new IllegalStateException( + "Execute mongo command failed " + execResult.getStdout()); + } + } catch (InterruptedException | IOException e) { + throw new IllegalStateException("Execute mongo command failed", e); + } + } + + @Override + protected void containerIsStarted(InspectContainerResponse containerInfo) { + log.info("Preparing a MongoDB Container with sharding cluster role {}...", clusterRole); + if (clusterRole != ShardingClusterRole.ROUTER) { + initReplicaSet(); + } else { + initShard(); + } + } + + protected void initReplicaSet() { + log.info("Initializing a single node replica set..."); + executeCommand( + String.format( + "rs.initiate({ _id : '%s', configsvr: %s, members: [{ _id: 0, host: '%s:%d'}]})", + clusterRole.replicaSetName, + clusterRole == ShardingClusterRole.CONFIG, + clusterRole.hostname, + MONGODB_PORT)); + + log.info("Waiting for single node replica set initialized..."); + executeCommand( + String.format( + "var attempt = 0; " + + "while" + + "(%s) " + + "{ " + + "if (attempt > %d) {quit(1);} " + + "print('%s ' + attempt); sleep(100); attempt++; " + + " }", + "db.runCommand( { isMaster: 1 } ).ismaster==false", + 60, + "An attempt to await for a single node replica set initialization:")); + } + + protected void initShard() { + log.info("Initializing a sharded cluster..."); + // decrease chunk size from default 64mb to 1mb to make splitter test easier. + executeCommand( + "db.getSiblingDB('config').settings.updateOne(\n" + + " { _id: \"chunksize\" },\n" + + " { $set: { _id: \"chunksize\", value: 1 } },\n" + + " { upsert: true }\n" + + ");"); + executeCommand( + String.format( + "sh.addShard('%s/%s:%d')", + ShardingClusterRole.SHARD.replicaSetName, + ShardingClusterRole.SHARD.hostname, + MONGODB_PORT)); + } + + public enum ShardingClusterRole { + // Config servers store metadata and configuration settings for the cluster. + CONFIG("config0", "rs0-config", Wait.forLogMessage(".*[Ww]aiting for connections.*", 2)), + + // Each shard contains a subset of the sharded data. Each shard can be deployed as a replica + // set. + SHARD("shard0", "rs0-shard", Wait.forLogMessage(".*[Ww]aiting for connections.*", 2)), + + // The mongos acts as a query router, providing an interface between client applications and + // the sharded cluster. + ROUTER("router0", null, Wait.forLogMessage(".*[Ww]aiting for connections.*", 1)), + + // None sharded cluster. + NONE("mongo0", "rs0", Wait.forLogMessage(".*Replication has not yet been configured.*", 1)); + + private final String hostname; + private final String replicaSetName; + private final WaitStrategy waitStrategy; + + ShardingClusterRole(String hostname, String replicaSetName, WaitStrategy waitStrategy) { + this.hostname = hostname; + this.replicaSetName = replicaSetName; + this.waitStrategy = waitStrategy; + } + + public static String startupCommand(ShardingClusterRole clusterRole) { + switch (clusterRole) { + case CONFIG: + return String.format( + "mongod --configsvr --port %d --replSet %s --keyFile /data/keyfile/random.key", + MONGODB_PORT, clusterRole.replicaSetName); + case SHARD: + return String.format( + "mongod --shardsvr --port %d --replSet %s --keyFile /data/keyfile/random.key", + MONGODB_PORT, clusterRole.replicaSetName); + case ROUTER: + return String.format( + "mongos --configdb %s/%s:%d --bind_ip_all --keyFile /data/keyfile/random.key", + CONFIG.replicaSetName, CONFIG.hostname, MONGODB_PORT); + case NONE: + default: + return String.format( + "mongod --port %d --replSet %s --keyFile /data/keyfile/random.key", + MONGODB_PORT, NONE.replicaSetName); + } + } + } + + public void executeCommandFileInSeparateDatabase(String fileNameIgnoreSuffix) { + executeCommandFileInDatabase(fileNameIgnoreSuffix, fileNameIgnoreSuffix); + } + + public void executeCommandFileInDatabase(String fileNameIgnoreSuffix, String databaseName) { + final String dbName = databaseName != null ? databaseName : fileNameIgnoreSuffix; + final String ddlFile = String.format("ddl/%s.js", fileNameIgnoreSuffix); + final URL ddlTestFile = MongoDBContainer.class.getClassLoader().getResource(ddlFile); + assertNotNull("Cannot locate " + ddlFile, ddlTestFile); + + try { + // use database; + String command0 = String.format("db = db.getSiblingDB('%s');\n", dbName); + String command1 = + Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() + .filter(x -> StringUtils.isNotBlank(x) && !x.trim().startsWith("//")) + .map( + x -> { + final Matcher m = COMMENT_PATTERN.matcher(x); + return m.matches() ? m.group(1) : x; + }) + .collect(Collectors.joining("\n")); + + executeCommand(command0 + command1); + + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java new file mode 100644 index 000000000000..8e42b6b8d4f7 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/java/mongodb/MongodbCDCIT.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package mongodb; + +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlContainer; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlVersion; +import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.UniqueDatabase; +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; + +import org.bson.Document; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.lifecycle.Startables; + +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.model.Sorts; +import lombok.extern.slf4j.Slf4j; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.await; + +@Slf4j +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "Currently SPARK and FLINK do not support cdc") +public class MongodbCDCIT extends TestSuiteBase implements TestResource { + + // ---------------------------------------------------------------------------- + // mongodb + protected static final String MONGODB_DATABASE = "inventory"; + + protected static final String MONGODB_COLLECTION = "products"; + protected MongoDBContainer mongodbContainer; + + protected MongoClient client; + + // ---------------------------------------------------------------------------- + // mysql + private static final String MYSQL_HOST = "mysql_cdc_e2e"; + + private static final String MYSQL_USER_NAME = "st_user"; + + private static final String MYSQL_USER_PASSWORD = "seatunnel"; + + private static final String MYSQL_DATABASE = "mongodb_cdc"; + + private static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(); + + // mysql sink table query sql + private static final String SINK_SQL = "select name,description,weight from products"; + + private final UniqueDatabase inventoryDatabase = + new UniqueDatabase(MYSQL_CONTAINER, MYSQL_DATABASE, "mysqluser", "mysqlpw"); + + private static MySqlContainer createMySqlContainer() { + MySqlContainer mySqlContainer = new MySqlContainer(MySqlVersion.V8_0); + mySqlContainer.withConfigurationOverride("docker/server-gtids/my.cnf"); + mySqlContainer.withSetupSQL("docker/setup.sql"); + mySqlContainer.withNetwork(NETWORK); + mySqlContainer.withNetworkAliases(MYSQL_HOST); + mySqlContainer.withDatabaseName(MYSQL_DATABASE); + mySqlContainer.withUsername(MYSQL_USER_NAME); + mySqlContainer.withPassword(MYSQL_USER_PASSWORD); + // For local test use + // mySqlContainer.setPortBindings(Collections.singletonList("3308:3306")); + return mySqlContainer; + } + + @BeforeAll + @Override + public void startUp() { + log.info("The first stage:Starting Mysql containers..."); + Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); + log.info("Mysql Containers are started"); + inventoryDatabase.createAndInitialize(); + log.info("Mysql ddl-a execution is complete"); + + log.info("The second stage:Starting Mongodb containers..."); + mongodbContainer = new MongoDBContainer(NETWORK); + // For local test use + // mongodbContainer.setPortBindings(Collections.singletonList("27017:27017")); + Startables.deepStart(Stream.of(mongodbContainer)).join(); + mongodbContainer.executeCommandFileInSeparateDatabase(MONGODB_DATABASE); + initConnection(); + log.info("Mongodb Container are started"); + } + + @TestTemplate + public void testMongodbCdcToMysqlCheckDataE2e(TestContainer container) { + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob("/mongodbcdc_to_mysql.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + readMongodbData().stream() + .peek(e -> e.remove("_id")) + .map(Document::entrySet) + .map(Set::stream) + .map( + entryStream -> + entryStream + .map(Map.Entry::getValue) + .collect( + Collectors.toCollection( + ArrayList + ::new))) + .collect(Collectors.toList()), + querySql()); + }); + + // insert update delete + upsertDeleteSourceTable(); + + await().atMost(60000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + readMongodbData().stream() + .peek(e -> e.remove("_id")) + .map(Document::entrySet) + .map(Set::stream) + .map( + entryStream -> + entryStream + .map(Map.Entry::getValue) + .collect( + Collectors.toCollection( + ArrayList + ::new))) + .collect(Collectors.toList()), + querySql()); + }); + } + + private Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection( + MYSQL_CONTAINER.getJdbcUrl(), + MYSQL_CONTAINER.getUsername(), + MYSQL_CONTAINER.getPassword()); + } + + private List> querySql() { + try (Connection connection = getJdbcConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(MongodbCDCIT.SINK_SQL); + List> result = new ArrayList<>(); + int columnCount = resultSet.getMetaData().getColumnCount(); + while (resultSet.next()) { + ArrayList objects = new ArrayList<>(); + for (int i = 1; i <= columnCount; i++) { + objects.add(resultSet.getObject(i)); + } + result.add(objects); + } + return result; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private void upsertDeleteSourceTable() { + mongodbContainer.executeCommandFileInDatabase("InventoryDDL", MONGODB_DATABASE); + } + + public void initConnection() { + String ipAddress = mongodbContainer.getHost(); + String url = + String.format( + "mongodb://%s:%s@%s:%d/%s?authSource=admin", + "stuser", "stpw", ipAddress, 27017, "inventory.products"); + client = MongoClients.create(url); + } + + protected List readMongodbData() { + MongoCollection sinkTable = + client.getDatabase(MONGODB_DATABASE).getCollection(MongodbCDCIT.MONGODB_COLLECTION); + MongoCursor cursor = sinkTable.find().sort(Sorts.ascending("_id")).cursor(); + List documents = new ArrayList<>(); + while (cursor.hasNext()) { + documents.add(cursor.next()); + } + return documents; + } + + @Override + @AfterAll + public void tearDown() { + // close Container + MYSQL_CONTAINER.close(); + if (mongodbContainer != null) { + mongodbContainer.close(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/InventoryDDL.js b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/InventoryDDL.js new file mode 100644 index 000000000000..db05f5f59fff --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/InventoryDDL.js @@ -0,0 +1,32 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// -- this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000109"), "name": "bicycle", "description": "Mountain bike with 21 gears", "weight": "1200"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000110"), "name": "headphones", "description": "Wireless headphones with noise cancellation", "weight": "200"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000111"), "name": "laptop", "description": "13-inch ultrabook with 16GB RAM and SSD storage", "weight": "1100"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000112"), "name": "blender", "description": "High-powered blender for smoothies and shakes", "weight": "400"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000113"), "name": "notebook", "description": "Spiral-bound notebook with ruled pages", "weight": "300"}); + +db.getCollection('products').updateOne({"name": "scooter"}, {$set: {"weight": "350"}}); +db.getCollection('products').updateOne({"name": "car battery"}, {$set: {"description": "High-performance car battery"}}); +db.getCollection('products').updateOne({"name": "12-pack drill bits"}, {$set: {"description": "Set of 12 professional-grade drill bits"}}); +db.getCollection('products').updateOne({"name": "hammer"}, {$set: {"weight": "100"}}); +db.getCollection('products').updateOne({"name": "rocks"}, {$set: {"weight": "1000"}}); + +db.getCollection('products').deleteOne({"_id": ObjectId("100000000000000000000101")}); +db.getCollection('products').deleteOne({"name": "car battery"}); +db.getCollection('products').deleteOne({"name": "12-pack drill bits"}); +db.getCollection('products').deleteOne({"name": "hammer", "weight": "875"}); +db.getCollection('products').deleteOne({"name": "jacket"}); \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventory.js b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventory.js new file mode 100644 index 000000000000..c834ec8a2cec --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/inventory.js @@ -0,0 +1,24 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// -- this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000101"), "name": "scooter", "description": "Small 2-wheel scooter", "weight": "314"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000102"), "name": "car battery", "description": "12V car battery", "weight": "81"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000103"), "name": "12-pack drill bits", "description": "12-pack of drill bits with sizes ranging from #40 to #3", "weight": "8"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000104"), "name": "hammer", "description": "12oz carpenter''s hammer", "weight": "75"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000105"), "name": "hammer", "description": "12oz carpenter''s hammer", "weight": "875"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000106"), "name": "hammer", "description": "12oz carpenter''s hammer", "weight": "10"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000107"), "name": "rocks", "description": "box of assorted rocks", "weight": "53"}); +db.getCollection('products').insertOne({"_id": ObjectId("100000000000000000000108"), "name": "jacket", "description": "water resistent black wind breaker", "weight": "1"}); + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/mongodb_cdc.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/mongodb_cdc.sql new file mode 100644 index 000000000000..69549a83a132 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/ddl/mongodb_cdc.sql @@ -0,0 +1,32 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: inventory +-- ---------------------------------------------------------------------------------------------------------------- +CREATE DATABASE IF NOT EXISTS `mongodb_cdc`; + +use mongodb_cdc; + +-- Create and populate our products using a single insert with many rows +CREATE TABLE products ( + _id VARCHAR(512) NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL, + description VARCHAR(512), + weight VARCHAR(255) +); + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/random.key b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/random.key new file mode 100644 index 000000000000..7398791e39d1 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/random.key @@ -0,0 +1,34 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +XK8G9pNKhEPp/BlsKT7pHEc5i0oCpvNVZMALH5pD/6EHSuMzuyO1FpoeDwmWHXl0 ++Gp+VOI89Xp7E6eqop+fFHtoM3Mnk2oTiI/442GvS0xISPTwFVY9nO3MfO8VcPVx +J3JCAb80GeXD5x55eAOi7NqXzpjk0OKqfPEwIn1lrjlkL2m5vq6kaKEd93i1+bMh +3LRd1jLbgwWWxqYVV92BTQNnJin+G1er7Y2FzLpeFIKqyy+I22qIE2XIC7yj3wSw +kxwKsPN5LjFsfVeKpf169R0KgBg4Nm0qlllVUGNKuEjaVoLOEBOJgoPnhC6L2avc +/iDeunZDlDDgYG6t6aJXJelP+W1uXp4JQj1j18Scn0lrvgWxdAVrAtK6ftxqutHc +RQBt6Ap63zojTraulm3aeo/w/yz0zjyYjxQ5t8cojIM/7TaNLe2GfVxwhqitUPL1 +ct2YFXWwX1H/+8E7yTsnquKqe6+r0aGQqxS5x+wFMsDun/1mxv7jgjwzZc1rEk8H +DGdhnQ7MFPOE6Bp03zGpa6B6K4I5uDgUUeOC7zmAN63cPEumuuCjPVK42sMt5wwR +NPJyL4+sWHa9vb2sBJ1dk3thQ+wwz856BZ9ILgeMUutQgasSwctlI7t3rhM+BGYy ++naEhKWN9/cIDXtl3ZMhNWJIh/MqbluYazQ/97MZHeWc9CJXFU6yUrnJOdE0VvQd +tROQNDuEB0Tq9ITxSYpZTY49+1CQp5E14GIc8frieWPvcbNVknriFquQfsW/tMvk +V2Aj8sBYE+sW9sGQJlyfRrhTSN6aBG1em7ZkOAgcx2/5ftaEZTwBxNnJR9VZDYEi +CDbobs3hIX3qhS6J9YbTEPFF2L6MMTL3ADgS44cWtmlYQrb2HJT0YLmdCzk4lSa6 +yWYLorduRtblgGo6v/nn7y41gn/l/aRdcDUsii/LgMco4ZPSRm0HixD8oA3agX9/ +23M5UVNCBO4/RKFOnjWM/2tN1xjeQrS2Hn6j3BtoTOl6k4ho diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/setup.js b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/setup.js new file mode 100644 index 000000000000..9cef6b876b95 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/mongodb/setup.js @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one or more +// contributor license agreements. See the NOTICE file distributed with +// -- this work for additional information regarding copyright ownership. +// The ASF licenses this file to You under the Apache License, Version 2.0 +// (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//use admin; +db.createRole( + { + role: "strole", + privileges: [{ + // Grant privileges on All Non-System Collections in All Databases + resource: {db: "", collection: ""}, + actions: ["splitVector", "listDatabases", "listCollections", "collStats", "find", "changeStream"] + }], + roles: [ + {role: 'read', db: 'config'} + ] + } +); + +db.createUser( + { + user: 'stuser', + pwd: 'stpw', + roles: [ + {role: 'strole', db: 'admin'} + ] + } +); \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/server-gtids/my.cnf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/server-gtids/my.cnf new file mode 100644 index 000000000000..a390897885d0 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/server-gtids/my.cnf @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# For advice on how to change settings please see +# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html + +[mysqld] +# +# Remove leading # and set to the amount of RAM for the most important data +# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. +# innodb_buffer_pool_size = 128M +# +# Remove leading # to turn on a very important data integrity option: logging +# changes to the binary log between backups. +# log_bin +# +# Remove leading # to set options mainly useful for reporting servers. +# The server defaults are faster for transactions and fast SELECTs. +# Adjust sizes as needed, experiment to find the optimal values. +# join_buffer_size = 128M +# sort_buffer_size = 2M +# read_rnd_buffer_size = 2M +skip-host-cache +skip-name-resolve +#datadir=/var/lib/mysql +#socket=/var/lib/mysql/mysql.sock +secure-file-priv=/var/lib/mysql +user=mysql + +# Disabling symbolic-links is recommended to prevent assorted security risks +symbolic-links=0 + +#log-error=/var/log/mysqld.log +#pid-file=/var/run/mysqld/mysqld.pid + +# ---------------------------------------------- +# Enable the binlog for replication & CDC +# ---------------------------------------------- + +# Enable binary replication log and set the prefix, expiration, and log format. +# The prefix is arbitrary, expiration can be short for integration tests but would +# be longer on a production system. Row-level info is required for ingest to work. +# Server ID is required, but this will vary on production systems +server-id = 223344 +log_bin = mysql-bin +expire_logs_days = 1 +binlog_format = row + +# enable gtid mode +gtid_mode = on +enforce_gtid_consistency = on \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/setup.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/setup.sql new file mode 100644 index 000000000000..2edd6c917e41 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/docker/setup.sql @@ -0,0 +1,32 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. +-- + +-- In production you would almost certainly limit the replication user must be on the follower (slave) machine, +-- to prevent other clients accessing the log from other machines. For example, 'replicator'@'follower.acme.com'. +-- However, in this database we'll grant 2 users different privileges: +-- +-- 1) 'st_user' - all privileges required by the snapshot reader AND binlog reader (used for testing) +-- 2) 'mysqluser' - all privileges +-- +GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT, LOCK TABLES ON *.* TO 'st_user'@'%'; +CREATE USER 'mysqluser' IDENTIFIED BY 'mysqlpw'; +GRANT ALL PRIVILEGES ON *.* TO 'mysqluser'@'%'; + +-- ---------------------------------------------------------------------------------------------------------------- +-- DATABASE: emptydb +-- ---------------------------------------------------------------------------------------------------------------- +CREATE DATABASE emptydb; diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/log4j2-test.properties b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/log4j2-test.properties new file mode 100644 index 000000000000..d1ca535f2be6 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/log4j2-test.properties @@ -0,0 +1,29 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level=INFO +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%t] %-5p %c - %m%n diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf new file mode 100644 index 000000000000..b4259756e3dc --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-mongodb-e2e/src/test/resources/mongodbcdc_to_mysql.conf @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + execution.checkpoint.interval = 5000 +} + +source { + MongoDB-CDC { + hosts = "mongo0:27017" + database = "inventory" + collection = "inventory.products" + username = stuser + password = stpw + schema = { + fields { + "_id": string, + "name": string, + "description": string, + "weight": string + } + } + } +} + +sink { + jdbc { + url = "jdbc:mysql://mysql_cdc_e2e:3306" + driver = "com.mysql.cj.jdbc.Driver" + user = "st_user" + password = "seatunnel" + + generate_sink_sql = true + # You need to configure both database and table + database = mongodb_cdc + table = products + primary_keys = ["_id"] + } +} \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 070ab6e936f5..bee396cbe939 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -47,6 +47,7 @@ connector-elasticsearch-e2e connector-iotdb-e2e connector-cdc-mysql-e2e + connector-cdc-mongodb-e2e connector-iceberg-e2e connector-iceberg-hadoop3-e2e connector-tdengine-e2e