|
18 | 18 |
|
19 | 19 | package com.ververica.cdc.connectors.mongodb.internal;
|
20 | 20 |
|
| 21 | +import com.mongodb.ConnectionString; |
| 22 | +import com.mongodb.MongoNamespace; |
| 23 | +import com.mongodb.client.MongoClient; |
| 24 | +import com.mongodb.client.MongoClients; |
| 25 | +import com.mongodb.kafka.connect.source.MongoSourceConfig; |
21 | 26 | import com.mongodb.kafka.connect.source.MongoSourceTask;
|
22 | 27 | import io.debezium.connector.AbstractSourceInfo;
|
23 | 28 | import io.debezium.connector.SnapshotRecord;
|
|
29 | 34 | import org.apache.kafka.connect.source.SourceRecord;
|
30 | 35 | import org.apache.kafka.connect.source.SourceTask;
|
31 | 36 | import org.apache.kafka.connect.source.SourceTaskContext;
|
| 37 | +import org.bson.conversions.Bson; |
32 | 38 | import org.bson.json.JsonReader;
|
33 | 39 |
|
34 | 40 | import java.lang.reflect.Field;
|
35 | 41 | import java.time.Instant;
|
| 42 | +import java.util.ArrayList; |
| 43 | +import java.util.Arrays; |
36 | 44 | import java.util.LinkedList;
|
37 | 45 | import java.util.List;
|
38 | 46 | import java.util.Map;
|
| 47 | +import java.util.Optional; |
39 | 48 | import java.util.concurrent.atomic.AtomicBoolean;
|
| 49 | +import java.util.regex.Pattern; |
| 50 | +import java.util.stream.Collectors; |
| 51 | + |
| 52 | +import static com.mongodb.client.model.Aggregates.match; |
| 53 | +import static com.mongodb.client.model.Filters.and; |
| 54 | +import static com.mongodb.client.model.Filters.regex; |
| 55 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.ADD_NS_FIELD; |
| 56 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.ADD_NS_FIELD_NAME; |
| 57 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.bsonListToJson; |
| 58 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.collectionNames; |
| 59 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.collectionsFilter; |
| 60 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.completionPattern; |
| 61 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.databaseFilter; |
| 62 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.databaseNames; |
| 63 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.includeListAsPatterns; |
| 64 | +import static com.ververica.cdc.connectors.mongodb.utils.CollectionDiscoveryUtils.isIncludeListExplicitlySpecified; |
40 | 65 |
|
41 | 66 | /**
|
42 | 67 | * Source Task that proxies mongodb kafka connector's {@link MongoSourceTask} to adapt to {@link
|
43 | 68 | * com.ververica.cdc.debezium.internal.DebeziumChangeFetcher}.
|
44 | 69 | */
|
45 | 70 | public class MongoDBConnectorSourceTask extends SourceTask {
|
46 | 71 |
|
| 72 | + public static final String DATABASE_INCLUDE_LIST = "database.include.list"; |
| 73 | + |
| 74 | + public static final String COLLECTION_INCLUDE_LIST = "collection.include.list"; |
| 75 | + |
47 | 76 | private static final String TRUE = "true";
|
48 | 77 |
|
49 | 78 | private static final Schema HEARTBEAT_VALUE_SCHEMA =
|
@@ -77,6 +106,7 @@ public void initialize(SourceTaskContext context) {
|
77 | 106 |
|
78 | 107 | @Override
|
79 | 108 | public void start(Map<String, String> props) {
|
| 109 | + initCapturedCollections(props); |
80 | 110 | target.start(props);
|
81 | 111 | isInSnapshotPhase = isCopying();
|
82 | 112 | }
|
@@ -226,4 +256,103 @@ private boolean isCopying() {
|
226 | 256 | throw new IllegalStateException("Cannot access isCopying field of SourceTask", e);
|
227 | 257 | }
|
228 | 258 | }
|
| 259 | + |
| 260 | + private void initCapturedCollections(Map<String, String> props) { |
| 261 | + ConnectionString connectionString = |
| 262 | + new ConnectionString(props.get(MongoSourceConfig.CONNECTION_URI_CONFIG)); |
| 263 | + |
| 264 | + String databaseIncludeList = props.get(DATABASE_INCLUDE_LIST); |
| 265 | + String collectionIncludeList = props.get(COLLECTION_INCLUDE_LIST); |
| 266 | + |
| 267 | + List<String> databaseList = |
| 268 | + Optional.ofNullable(databaseIncludeList) |
| 269 | + .map(input -> Arrays.asList(input.split(","))) |
| 270 | + .orElse(null); |
| 271 | + |
| 272 | + List<String> collectionList = |
| 273 | + Optional.ofNullable(collectionIncludeList) |
| 274 | + .map(input -> Arrays.asList(input.split(","))) |
| 275 | + .orElse(null); |
| 276 | + |
| 277 | + if (collectionList != null) { |
| 278 | + // Watching collections changes |
| 279 | + List<String> discoveredDatabases; |
| 280 | + List<String> discoveredCollections; |
| 281 | + try (MongoClient mongoClient = MongoClients.create(connectionString)) { |
| 282 | + discoveredDatabases = databaseNames(mongoClient, databaseFilter(databaseList)); |
| 283 | + discoveredCollections = |
| 284 | + collectionNames( |
| 285 | + mongoClient, |
| 286 | + discoveredDatabases, |
| 287 | + collectionsFilter(collectionList)); |
| 288 | + } |
| 289 | + |
| 290 | + // case: database = db0, collection = coll1 |
| 291 | + if (isIncludeListExplicitlySpecified(collectionList, discoveredCollections)) { |
| 292 | + MongoNamespace namespace = new MongoNamespace(discoveredCollections.get(0)); |
| 293 | + props.put(MongoSourceConfig.DATABASE_CONFIG, namespace.getDatabaseName()); |
| 294 | + props.put(MongoSourceConfig.COLLECTION_CONFIG, namespace.getCollectionName()); |
| 295 | + } else { // case: database = db0|db2, collection = (db0.coll[0-9])|(db1.coll[1-2]) |
| 296 | + String namespacesRegex = |
| 297 | + includeListAsPatterns(collectionList).stream() |
| 298 | + .map(Pattern::pattern) |
| 299 | + .collect(Collectors.joining("|")); |
| 300 | + |
| 301 | + List<Bson> pipeline = new ArrayList<>(); |
| 302 | + pipeline.add(ADD_NS_FIELD); |
| 303 | + |
| 304 | + Bson nsFilter = regex(ADD_NS_FIELD_NAME, namespacesRegex); |
| 305 | + if (databaseList != null) { |
| 306 | + String databaseRegex = |
| 307 | + includeListAsPatterns(databaseList).stream() |
| 308 | + .map(Pattern::pattern) |
| 309 | + .collect(Collectors.joining("|")); |
| 310 | + Bson dbFilter = regex("ns.db", databaseRegex); |
| 311 | + nsFilter = and(dbFilter, nsFilter); |
| 312 | + } |
| 313 | + pipeline.add(match(nsFilter)); |
| 314 | + |
| 315 | + props.put(MongoSourceConfig.PIPELINE_CONFIG, bsonListToJson(pipeline)); |
| 316 | + |
| 317 | + String copyExistingNamespaceRegex = |
| 318 | + discoveredCollections.stream() |
| 319 | + .map(ns -> completionPattern(ns).pattern()) |
| 320 | + .collect(Collectors.joining("|")); |
| 321 | + |
| 322 | + props.put( |
| 323 | + MongoSourceConfig.COPY_EXISTING_NAMESPACE_REGEX_CONFIG, |
| 324 | + copyExistingNamespaceRegex); |
| 325 | + } |
| 326 | + } else if (databaseList != null) { |
| 327 | + // Watching databases changes |
| 328 | + List<String> discoveredDatabases; |
| 329 | + try (MongoClient mongoClient = MongoClients.create(connectionString)) { |
| 330 | + discoveredDatabases = databaseNames(mongoClient, databaseFilter(databaseList)); |
| 331 | + } |
| 332 | + |
| 333 | + if (isIncludeListExplicitlySpecified(databaseList, discoveredDatabases)) { |
| 334 | + props.put(MongoSourceConfig.DATABASE_CONFIG, discoveredDatabases.get(0)); |
| 335 | + } else { |
| 336 | + String databaseRegex = |
| 337 | + includeListAsPatterns(databaseList).stream() |
| 338 | + .map(Pattern::pattern) |
| 339 | + .collect(Collectors.joining("|")); |
| 340 | + |
| 341 | + List<Bson> pipeline = new ArrayList<>(); |
| 342 | + pipeline.add(match(regex("ns.db", databaseRegex))); |
| 343 | + props.put(MongoSourceConfig.PIPELINE_CONFIG, bsonListToJson(pipeline)); |
| 344 | + |
| 345 | + String copyExistingNamespaceRegex = |
| 346 | + discoveredDatabases.stream() |
| 347 | + .map(db -> completionPattern(db + "\\..*").pattern()) |
| 348 | + .collect(Collectors.joining("|")); |
| 349 | + |
| 350 | + props.put( |
| 351 | + MongoSourceConfig.COPY_EXISTING_NAMESPACE_REGEX_CONFIG, |
| 352 | + copyExistingNamespaceRegex); |
| 353 | + } |
| 354 | + } else { |
| 355 | + // Watching all changes on the cluster by default, we do nothing here |
| 356 | + } |
| 357 | + } |
229 | 358 | }
|
0 commit comments