From 7ba91177478ee09b71f273e3b8f551ad850b8005 Mon Sep 17 00:00:00 2001 From: Jiabao Sun Date: Mon, 19 Jun 2023 00:03:29 +0800 Subject: [PATCH] [FLINK-32344][connectors/mongodb] Support unbounded streaming read via ChangeStream feature. --- .../tests/util/mongodb/MongoE2ECase.java | 98 ++++- .../src/test/resources/e2e_unbounded.sql | 47 +++ .../mongodb/common/utils/MongoConstants.java | 20 + .../mongodb/common/utils/MongoUtils.java | 91 +++- .../mongodb/sink/writer/MongoWriter.java | 4 +- .../connector/mongodb/source/MongoSource.java | 72 +++- .../mongodb/source/MongoSourceBuilder.java | 67 +++ .../config/MongoChangeStreamOptions.java | 157 +++++++ .../source/config/MongoReadOptions.java | 44 ++ .../source/config/MongoStartupOptions.java | 131 ++++++ .../enumerator/MongoSourceEnumState.java | 47 ++- .../MongoSourceEnumStateSerializer.java | 17 +- .../enumerator/MongoSourceEnumerator.java | 20 +- .../assigner/MongoHybridSplitAssigner.java | 135 ++++++ .../assigner/MongoScanSplitAssigner.java | 14 +- .../assigner/MongoSplitAssigner.java | 10 +- .../assigner/MongoStreamSplitAssigner.java | 112 +++++ .../source/reader/MongoSourceReader.java | 23 +- .../source/reader/MongoSourceRecord.java | 75 ++++ .../MongoDeserializationSchema.java | 26 +- .../reader/emitter/MongoRecordEmitter.java | 31 +- .../split/MongoHybridSourceSplitReader.java | 140 +++++++ .../split/MongoScanSourceSplitReader.java | 50 +-- .../reader/split/MongoSourceSplitReader.java | 15 +- .../split/MongoStreamSourceSplitReader.java | 254 +++++++++++ .../source/split/MongoScanSourceSplit.java | 20 + .../split/MongoScanSourceSplitState.java | 5 +- .../split/MongoSourceSplitSerializer.java | 44 +- .../source/split/MongoSourceSplitState.java | 5 +- .../source/split/MongoStreamOffset.java | 118 ++++++ .../source/split/MongoStreamSourceSplit.java | 94 +++++ .../split/MongoStreamSourceSplitState.java | 62 +++ .../mongodb/table/MongoConnectorOptions.java | 68 +++ .../table/MongoDynamicTableFactory.java | 40 ++ .../table/MongoDynamicTableSource.java | 35 +- .../table/MongoRowDataLookupFunction.java | 4 +- .../table/config/FullDocumentStrategy.java | 66 +++ .../table/config/MongoConfiguration.java | 37 ++ .../MongoRowDataDeserializationSchema.java | 101 ++++- .../mongodb/source/MongoSourceITCase.java | 11 +- .../source/MongoUnboundedSourceITCase.java | 393 ++++++++++++++++++ .../MongoSourceEnumStateSerializerTest.java | 5 +- .../split/MongoSourceSplitSerializerTest.java | 67 +++ .../table/MongoDynamicTableFactoryTest.java | 79 ++++ .../table/MongoDynamicTableSourceITCase.java | 299 +++++++------ .../table/MongoLookupTableSourceITCase.java | 295 +++++++++++++ .../testutils/MongoShardedContainers.java | 52 ++- .../mongodb/testutils/MongoTestUtil.java | 32 +- pom.xml | 9 +- 49 files changed, 3305 insertions(+), 336 deletions(-) create mode 100644 flink-connector-mongodb-e2e-tests/src/test/resources/e2e_unbounded.sql create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoChangeStreamOptions.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoStartupOptions.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoHybridSplitAssigner.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoStreamSplitAssigner.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceRecord.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoHybridSourceSplitReader.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoStreamSourceSplitReader.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamOffset.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamSourceSplit.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamSourceSplitState.java create mode 100644 flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/FullDocumentStrategy.java create mode 100644 flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoUnboundedSourceITCase.java create mode 100644 flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitSerializerTest.java create mode 100644 flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoLookupTableSourceITCase.java diff --git a/flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java b/flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java index 082f3128..a1e0b67c 100644 --- a/flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java +++ b/flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java @@ -22,12 +22,16 @@ import org.apache.flink.connector.testframe.container.FlinkContainers; import org.apache.flink.connector.testframe.container.FlinkContainersSettings; import org.apache.flink.connector.testframe.container.TestcontainersSettings; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.test.resources.ResourceTestUtils; import org.apache.flink.test.util.SQLJobSubmission; import com.mongodb.client.MongoClient; import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.Filters; +import com.mongodb.client.model.Updates; import org.bson.Document; import org.bson.types.ObjectId; import org.junit.jupiter.api.AfterAll; @@ -47,11 +51,13 @@ import java.nio.file.Paths; import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGODB_HOSTNAME; import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_4_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_IMAGE_PREFIX; import static org.assertj.core.api.Assertions.assertThat; /** End-to-end test for the MongoDB connectors. */ @@ -65,11 +71,11 @@ class MongoE2ECase { private static final Path SQL_CONNECTOR_MONGODB_JAR = ResourceTestUtils.getResource(".*mongodb.jar"); - private static final int TEST_ORDERS_COUNT = 5; + private static final int TEST_ORDERS_INITIAL_COUNT = 5; @Container static final MongoDBContainer MONGO_CONTAINER = - new MongoDBContainer(MONGO_4_0) + new MongoDBContainer(MONGO_IMAGE_PREFIX + MONGO_4_0) .withLogConsumer(new Slf4jLogConsumer(LOG)) .withNetwork(NETWORK) .withNetworkAliases(MONGODB_HOSTNAME); @@ -85,7 +91,12 @@ class MongoE2ECase { public static final FlinkContainers FLINK = FlinkContainers.builder() .withFlinkContainersSettings( - FlinkContainersSettings.builder().numTaskManagers(2).build()) + FlinkContainersSettings.builder() + .setConfigOption( + ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL, + Duration.ofSeconds(1)) + .numTaskManagers(2) + .build()) .withTestcontainersSettings(TESTCONTAINERS_SETTINGS) .build(); @@ -107,12 +118,12 @@ static void teardown() { public void testUpsertSink() throws Exception { MongoDatabase db = mongoClient.getDatabase("test_upsert"); - List orders = mockOrders(); + List orders = generateOrders(); db.getCollection("orders").insertMany(orders); executeSqlStatements(readSqlFile("e2e_upsert.sql")); - List ordersBackup = readAllBackupOrders(db); + List ordersBackup = readAllBackupOrders(db, orders.size()); assertThat(ordersBackup).containsExactlyInAnyOrderElementsOf(orders); } @@ -121,43 +132,88 @@ public void testUpsertSink() throws Exception { public void testAppendOnlySink() throws Exception { MongoDatabase db = mongoClient.getDatabase("test_append_only"); - List orders = mockOrders(); + List orders = generateOrders(); db.getCollection("orders").insertMany(orders); executeSqlStatements(readSqlFile("e2e_append_only.sql")); - List ordersBackup = readAllBackupOrders(db); + List ordersBackup = readAllBackupOrders(db, orders.size()); List expected = removeIdField(orders); assertThat(removeIdField(ordersBackup)).containsExactlyInAnyOrderElementsOf(expected); } - private static List readAllBackupOrders(MongoDatabase db) throws Exception { - Deadline deadline = Deadline.fromNow(Duration.ofSeconds(20)); - List backupOrders; - do { - Thread.sleep(1000); - backupOrders = db.getCollection("orders_bak").find().into(new ArrayList<>()); - } while (deadline.hasTimeLeft() && backupOrders.size() < TEST_ORDERS_COUNT); + @Test + public void testUnboundedSink() throws Exception { + MongoDatabase db = mongoClient.getDatabase("test_unbounded"); + MongoCollection coll = db.getCollection("orders"); + + List orders = generateOrders(); + coll.insertMany(orders); + + executeSqlStatements(readSqlFile("e2e_unbounded.sql")); + + // -- scan records -- + List ordersBackup = readAllBackupOrders(db, orders.size()); + assertThat(ordersBackup).containsExactlyInAnyOrderElementsOf(orders); + + // -- stream records -- + // insert 3 records + List newOrders = + Arrays.asList(generateOrder(6), generateOrder(7), generateOrder(8)); + coll.insertMany(newOrders); + orders.addAll(newOrders); + + // assert inserted + ordersBackup = readAllBackupOrders(db, orders.size()); + assertThat(ordersBackup).containsExactlyInAnyOrderElementsOf(orders); - return backupOrders; + // update 1 record + Document updateOrder = orders.get(0); + coll.updateOne(Filters.eq("_id", updateOrder.get("_id")), Updates.set("quantity", 1000L)); + + // replace 1 record + Document replacement = Document.parse(orders.get(1).toJson()); + replacement.put("quantity", 1001L); + coll.replaceOne(Filters.eq("_id", replacement.remove("_id")), replacement); + + // asert updated + ordersBackup = readAllBackupOrders(db, orders.size()); + assertThat(ordersBackup).containsExactlyInAnyOrderElementsOf(orders); + } + + private static List readAllBackupOrders(MongoDatabase db, int expectSize) + throws Exception { + Deadline deadline = Deadline.fromNow(Duration.ofSeconds(30)); + MongoCollection coll = db.getCollection("orders_bak"); + while (deadline.hasTimeLeft()) { + if (coll.countDocuments() < expectSize) { + Thread.sleep(1000L); + } else { + break; + } + } + return coll.find().into(new ArrayList<>()); } private static List removeIdField(List documents) { return documents.stream().peek(doc -> doc.remove("_id")).collect(Collectors.toList()); } - private static List mockOrders() { + private static List generateOrders() { List orders = new ArrayList<>(); - for (int i = 1; i <= TEST_ORDERS_COUNT; i++) { - orders.add( - new Document("_id", new ObjectId()) - .append("code", "ORDER_" + i) - .append("quantity", i * 10L)); + for (int i = 1; i <= TEST_ORDERS_INITIAL_COUNT; i++) { + orders.add(generateOrder(i)); } return orders; } + private static Document generateOrder(int index) { + return new Document("_id", new ObjectId()) + .append("code", "ORDER_" + index) + .append("quantity", index * 10L); + } + private static List readSqlFile(final String resourceName) throws Exception { return Files.readAllLines( Paths.get(MongoE2ECase.class.getResource("/" + resourceName).toURI())); diff --git a/flink-connector-mongodb-e2e-tests/src/test/resources/e2e_unbounded.sql b/flink-connector-mongodb-e2e-tests/src/test/resources/e2e_unbounded.sql new file mode 100644 index 00000000..c7c42f68 --- /dev/null +++ b/flink-connector-mongodb-e2e-tests/src/test/resources/e2e_unbounded.sql @@ -0,0 +1,47 @@ +--/* +-- * 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. +-- */ + +DROP TABLE IF EXISTS orders; +DROP TABLE IF EXISTS orders_bak; + +CREATE TABLE orders ( + `_id` STRING, + `code` STRING, + `quantity` BIGINT, + PRIMARY KEY (_id) NOT ENFORCED +) WITH ( + 'connector' = 'mongodb', + 'uri' = 'mongodb://mongodb:27017', + 'database' = 'test_unbounded', + 'collection' = 'orders', + 'scan.startup.mode' = 'initial' +); + +CREATE TABLE orders_bak ( + `_id` STRING, + `code` STRING, + `quantity` BIGINT, + PRIMARY KEY (_id) NOT ENFORCED +) WITH ( + 'connector' = 'mongodb', + 'uri' = 'mongodb://mongodb:27017', + 'database' = 'test_unbounded', + 'collection' = 'orders_bak' +); + +INSERT INTO orders_bak SELECT * FROM orders; diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoConstants.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoConstants.java index e60b4532..ccb7e321 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoConstants.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoConstants.java @@ -59,6 +59,18 @@ public class MongoConstants { public static final String DROPPED_FIELD = "dropped"; + public static final String CLUSTER_TIME_FIELD = "clusterTime"; + + public static final String RESUME_TOKEN_FIELD = "resumeToken"; + + public static final String OPERATION_TYPE_FIELD = "operationType"; + + public static final String DOCUMENT_KEY_FIELD = "documentKey"; + + public static final String FULL_DOCUMENT_FIELD = "fullDocument"; + + public static final String FULL_DOCUMENT_BEFORE_CHANGE_FIELD = "fullDocumentBeforeChange"; + public static final BsonValue BSON_MIN_KEY = new BsonMinKey(); public static final BsonValue BSON_MAX_KEY = new BsonMaxKey(); @@ -68,5 +80,13 @@ public class MongoConstants { public static final JsonWriterSettings DEFAULT_JSON_WRITER_SETTINGS = JsonWriterSettings.builder().outputMode(JsonMode.EXTENDED).build(); + 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; + private MongoConstants() {} } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java index 6f168b88..df1c680c 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java @@ -18,21 +18,34 @@ package org.apache.flink.connector.mongodb.common.utils; import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; +import org.apache.flink.connector.mongodb.source.split.MongoStreamOffset; import com.mongodb.MongoNamespace; +import com.mongodb.client.ChangeStreamIterable; +import com.mongodb.client.MongoChangeStreamCursor; import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.changestream.ChangeStreamDocument; +import org.apache.commons.lang3.StringUtils; import org.bson.BsonBoolean; 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.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.function.Function; import static com.mongodb.client.model.Filters.eq; import static com.mongodb.client.model.Filters.or; @@ -40,6 +53,7 @@ import static com.mongodb.client.model.Projections.fields; import static com.mongodb.client.model.Projections.include; import static com.mongodb.client.model.Sorts.ascending; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.CLUSTER_TIME_FIELD; import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.DROPPED_FIELD; import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD; import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD; @@ -49,21 +63,36 @@ import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD; import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD; -/** A util class with some helper method for MongoDB commands. */ +/** An util class with some helper method for MongoDB commands. */ @Internal public class MongoUtils { + private static final Logger LOG = LoggerFactory.getLogger(MongoUtils.class); + private static final String COLL_STATS_COMMAND = "collStats"; private static final String SPLIT_VECTOR_COMMAND = "splitVector"; private static final String KEY_PATTERN_OPTION = "keyPattern"; private static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize"; + private static final String IS_MASTER_COMMAND = "isMaster"; + private static final String ADMIN_DATABASE = "admin"; private static final String CONFIG_DATABASE = "config"; private static final String COLLECTIONS_COLLECTION = "collections"; private static final String CHUNKS_COLLECTION = "chunks"; private MongoUtils() {} + public static MongoClient clientFor(MongoConnectionOptions connectionOptions) { + return MongoClients.create(connectionOptions.getUri()); + } + + public static T doWithMongoClient( + MongoConnectionOptions connectionOptions, Function action) { + try (MongoClient mongoClient = clientFor(connectionOptions)) { + return action.apply(mongoClient); + } + } + public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) { BsonDocument collStatsCommand = new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName())); @@ -148,4 +177,64 @@ public static Bson project(List projectedFields) { return fields(include(projectedFields), excludeId()); } } + + public static MongoStreamOffset displayCurrentOffset(MongoConnectionOptions connectionOptions) { + return doWithMongoClient( + connectionOptions, + client -> { + ChangeStreamIterable changeStreamIterable = + getChangeStreamIterable( + client, + connectionOptions.getDatabase(), + connectionOptions.getCollection()); + + try (MongoChangeStreamCursor> + changeStreamCursor = changeStreamIterable.cursor()) { + ChangeStreamDocument firstResult = changeStreamCursor.tryNext(); + BsonDocument resumeToken = + firstResult != null + ? firstResult.getResumeToken() + : changeStreamCursor.getResumeToken(); + + // Nullable when no change record or postResumeToken (new in MongoDB 4.0.7). + return Optional.ofNullable(resumeToken) + .map(MongoStreamOffset::fromResumeToken) + .orElse( + MongoStreamOffset.fromClusterTime( + currentClusterTime(client))); + } + }); + } + + public static BsonTimestamp currentClusterTime(MongoClient mongoClient) { + return isMaster(mongoClient) + .getDocument("$" + CLUSTER_TIME_FIELD) + .getTimestamp(CLUSTER_TIME_FIELD); + } + + public static BsonDocument isMaster(MongoClient mongoClient) { + BsonDocument isMasterCommand = new BsonDocument(IS_MASTER_COMMAND, new BsonInt32(1)); + return mongoClient + .getDatabase(ADMIN_DATABASE) + .runCommand(isMasterCommand, BsonDocument.class); + } + + public static ChangeStreamIterable getChangeStreamIterable( + MongoClient mongoClient, @Nullable String database, @Nullable String collection) { + 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)) { + MongoDatabase db = mongoClient.getDatabase(database); + LOG.info("Preparing change stream for database {}", database); + changeStream = db.watch(); + } else { + LOG.info("Preparing change stream for deployment"); + changeStream = mongoClient.watch(); + } + return changeStream; + } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java index 5319959d..736eec41 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java @@ -37,7 +37,6 @@ import com.mongodb.MongoException; import com.mongodb.client.MongoClient; -import com.mongodb.client.MongoClients; import com.mongodb.client.model.WriteModel; import org.bson.BsonDocument; import org.slf4j.Logger; @@ -51,6 +50,7 @@ import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; +import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.clientFor; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -118,7 +118,7 @@ public MongoWriter( } // Initialize the mongo client. - this.mongoClient = MongoClients.create(connectionOptions.getUri()); + this.mongoClient = clientFor(connectionOptions); boolean flushOnlyOnCheckpoint = batchIntervalMs == -1 && batchSize == -1; diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSource.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSource.java index 6b1b89cb..ec0b272d 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSource.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSource.java @@ -30,22 +30,26 @@ import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; +import org.apache.flink.connector.mongodb.source.config.MongoChangeStreamOptions; import org.apache.flink.connector.mongodb.source.config.MongoReadOptions; +import org.apache.flink.connector.mongodb.source.config.MongoStartupOptions; import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumState; import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumStateSerializer; import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumerator; +import org.apache.flink.connector.mongodb.source.enumerator.assigner.MongoHybridSplitAssigner; import org.apache.flink.connector.mongodb.source.enumerator.assigner.MongoScanSplitAssigner; import org.apache.flink.connector.mongodb.source.enumerator.assigner.MongoSplitAssigner; +import org.apache.flink.connector.mongodb.source.enumerator.assigner.MongoStreamSplitAssigner; import org.apache.flink.connector.mongodb.source.reader.MongoSourceReader; import org.apache.flink.connector.mongodb.source.reader.MongoSourceReaderContext; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema; import org.apache.flink.connector.mongodb.source.reader.emitter.MongoRecordEmitter; -import org.apache.flink.connector.mongodb.source.reader.split.MongoScanSourceSplitReader; +import org.apache.flink.connector.mongodb.source.reader.split.MongoHybridSourceSplitReader; import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit; import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitSerializer; import org.apache.flink.core.io.SimpleVersionedSerializer; - -import org.bson.BsonDocument; +import org.apache.flink.table.api.ValidationException; import javax.annotation.Nullable; @@ -84,31 +88,35 @@ public class MongoSource /** The read options for MongoDB source. */ private final MongoReadOptions readOptions; + /** The change stream options for MongoDB source. */ + private final MongoChangeStreamOptions changeStreamOptions; + + /** The startup options for MongoDB source. */ + private final MongoStartupOptions startupOptions; + /** The projections for MongoDB source. */ @Nullable private final List projectedFields; /** The limit for MongoDB source. */ private final int limit; - /** The boundedness for MongoDB source. */ - private final Boundedness boundedness; - /** The mongo deserialization schema used for deserializing message. */ private final MongoDeserializationSchema deserializationSchema; MongoSource( MongoConnectionOptions connectionOptions, MongoReadOptions readOptions, + MongoChangeStreamOptions changeStreamOptions, + MongoStartupOptions startupOptions, @Nullable List projectedFields, int limit, MongoDeserializationSchema deserializationSchema) { this.connectionOptions = checkNotNull(connectionOptions); this.readOptions = checkNotNull(readOptions); + this.changeStreamOptions = checkNotNull(changeStreamOptions); + this.startupOptions = checkNotNull(startupOptions); this.projectedFields = projectedFields; this.limit = limit; - // Only support bounded mode for now. - // We can implement unbounded mode by ChangeStream future. - this.boundedness = Boundedness.BOUNDED; this.deserializationSchema = checkNotNull(deserializationSchema); } @@ -123,24 +131,25 @@ public static MongoSourceBuilder builder() { @Override public Boundedness getBoundedness() { - return boundedness; + return startupOptions.boundedness(); } @Override public SourceReader createReader(SourceReaderContext readerContext) { - FutureCompletingBlockingQueue> elementsQueue = + FutureCompletingBlockingQueue> elementsQueue = new FutureCompletingBlockingQueue<>(); MongoSourceReaderContext mongoReaderContext = new MongoSourceReaderContext(readerContext, limit); - Supplier> splitReaderSupplier = + Supplier> splitReaderSupplier = () -> - new MongoScanSourceSplitReader( + new MongoHybridSourceSplitReader( connectionOptions, readOptions, - projectedFields, - mongoReaderContext); + changeStreamOptions, + mongoReaderContext, + projectedFields); return new MongoSourceReader<>( elementsQueue, @@ -153,17 +162,15 @@ public SourceReader createReader(SourceReaderContext read public SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext) { MongoSourceEnumState initialState = MongoSourceEnumState.initialState(); - MongoSplitAssigner splitAssigner = - new MongoScanSplitAssigner(connectionOptions, readOptions, initialState); - return new MongoSourceEnumerator(boundedness, enumContext, splitAssigner); + return new MongoSourceEnumerator( + startupOptions.boundedness(), enumContext, createMongoSplitAssigner(initialState)); } @Override public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, MongoSourceEnumState checkpoint) { - MongoSplitAssigner splitAssigner = - new MongoScanSplitAssigner(connectionOptions, readOptions, checkpoint); - return new MongoSourceEnumerator(boundedness, enumContext, splitAssigner); + return new MongoSourceEnumerator( + startupOptions.boundedness(), enumContext, createMongoSplitAssigner(checkpoint)); } @Override @@ -180,4 +187,27 @@ public SimpleVersionedSerializer getEnumeratorCheckpointSe public TypeInformation getProducedType() { return deserializationSchema.getProducedType(); } + + private MongoSplitAssigner createMongoSplitAssigner(MongoSourceEnumState sourceEnumState) { + MongoSplitAssigner splitAssigner; + switch (startupOptions.getStartupMode()) { + case BOUNDED: + splitAssigner = + new MongoScanSplitAssigner(connectionOptions, readOptions, sourceEnumState); + break; + case LATEST_OFFSET: + case TIMESTAMP: + splitAssigner = new MongoStreamSplitAssigner(connectionOptions, startupOptions); + break; + case INITIAL: + splitAssigner = + new MongoHybridSplitAssigner( + connectionOptions, readOptions, sourceEnumState); + break; + default: + throw new ValidationException( + "Unsupported startup mode " + startupOptions.getStartupMode()); + } + return splitAssigner; + } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSourceBuilder.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSourceBuilder.java index c5252eb2..dd8ccbfc 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSourceBuilder.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSourceBuilder.java @@ -20,11 +20,15 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.MemorySize; import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; +import org.apache.flink.connector.mongodb.source.config.MongoChangeStreamOptions; import org.apache.flink.connector.mongodb.source.config.MongoReadOptions; +import org.apache.flink.connector.mongodb.source.config.MongoStartupOptions; import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy; import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema; import org.apache.flink.connector.mongodb.source.reader.split.MongoScanSourceSplitReader; +import com.mongodb.client.model.changestream.FullDocument; +import com.mongodb.client.model.changestream.FullDocumentBeforeChange; import org.bson.BsonDocument; import java.util.Arrays; @@ -45,7 +49,10 @@ public class MongoSourceBuilder { private final MongoConnectionOptions.MongoConnectionOptionsBuilder connectionOptionsBuilder; private final MongoReadOptions.MongoReadOptionsBuilder readOptionsBuilder; + private final MongoChangeStreamOptions.MongoChangeStreamOptionsBuilder + changeStreamOptionsBuilder; + private MongoStartupOptions startupOptions = MongoStartupOptions.bounded(); private List projectedFields; private int limit = -1; private MongoDeserializationSchema deserializationSchema; @@ -53,6 +60,7 @@ public class MongoSourceBuilder { MongoSourceBuilder() { this.connectionOptionsBuilder = MongoConnectionOptions.builder(); this.readOptionsBuilder = MongoReadOptions.builder(); + this.changeStreamOptionsBuilder = MongoChangeStreamOptions.builder(); } /** @@ -88,6 +96,17 @@ public MongoSourceBuilder setCollection(String collection) { return this; } + /** + * Specifies the startup options. + * + * @param startupOptions the connector startup options {@link MongoStartupOptions}. + * @return this builder + */ + public MongoSourceBuilder setStartupOptions(MongoStartupOptions startupOptions) { + this.startupOptions = startupOptions; + return this; + } + /** * Sets the number of documents should be fetched per round-trip when reading. * @@ -192,6 +211,52 @@ public MongoSourceBuilder setProjectedFields(List projectedFields) return this; } + /** + * Sets the number of change stream documents should be fetched per round-trip when reading. + * + * @param changeStreamFetchSize the number of change stream documents should be fetched per + * round-trip when reading. + * @return this builder + */ + public MongoSourceBuilder setChangeStreamFetchSize(int changeStreamFetchSize) { + changeStreamOptionsBuilder.setFetchSize(changeStreamFetchSize); + return this; + } + + /** + * Determines what values your change stream returns on update operations. The default setting + * returns the differences between the original document and the updated document. The + * updateLookup setting returns the differences between the original document and updated + * document as well as a copy of the entire updated document at a point in time after the + * update. The whenAvailable setting returns the updated document, if available. The required + * setting returns the updated document and raises an error if it is not available. + * + * @param fullDocument the values your change stream returns on update operations. + * @return this builder + */ + public MongoSourceBuilder setFullDocument(FullDocument fullDocument) { + changeStreamOptionsBuilder.setFullDocument(fullDocument); + return this; + } + + /** + * Configures the document pre-image your change stream returns on update operations. The + * pre-image is not available for source records published while copying existing data, and the + * pre-image configuration has no effect on copying. The default setting suppresses the document + * pre-image. The whenAvailable setting returns the document pre-image if it's available, before + * it was replaced, updated, or deleted. The required setting returns the document pre-image and + * raises an error if it is not available. + * + * @param fullDocumentBeforeChange the document pre-image your change stream returns on update + * operations. + * @return this builder + */ + public MongoSourceBuilder setFullDocumentBeforeChange( + FullDocumentBeforeChange fullDocumentBeforeChange) { + changeStreamOptionsBuilder.setFullDocumentBeforeChange(fullDocumentBeforeChange); + return this; + } + /** * Sets the deserialization schema for MongoDB {@link BsonDocument}. * @@ -215,6 +280,8 @@ public MongoSource build() { return new MongoSource<>( connectionOptionsBuilder.build(), readOptionsBuilder.build(), + changeStreamOptionsBuilder.build(), + startupOptions, projectedFields, limit, deserializationSchema); diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoChangeStreamOptions.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoChangeStreamOptions.java new file mode 100644 index 00000000..0990eb11 --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoChangeStreamOptions.java @@ -0,0 +1,157 @@ +/* + * 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.flink.connector.mongodb.source.config; + +import org.apache.flink.annotation.PublicEvolving; + +import com.mongodb.client.model.changestream.FullDocument; +import com.mongodb.client.model.changestream.FullDocumentBeforeChange; + +import java.io.Serializable; +import java.util.Objects; + +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.CHANGE_STREAM_FETCH_SIZE; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The change stream configuration class for MongoDB source. */ +@PublicEvolving +public class MongoChangeStreamOptions implements Serializable { + + private static final long serialVersionUID = 1L; + + private final int fetchSize; + + private final FullDocument fullDocument; + + private final FullDocumentBeforeChange fullDocumentBeforeChange; + + private MongoChangeStreamOptions( + int fetchSize, + FullDocument fullDocument, + FullDocumentBeforeChange fullDocumentBeforeChange) { + this.fetchSize = fetchSize; + this.fullDocument = fullDocument; + this.fullDocumentBeforeChange = fullDocumentBeforeChange; + } + + public int getFetchSize() { + return fetchSize; + } + + public FullDocument getFullDocument() { + return fullDocument; + } + + public FullDocumentBeforeChange getFullDocumentBeforeChange() { + return fullDocumentBeforeChange; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MongoChangeStreamOptions that = (MongoChangeStreamOptions) o; + return fetchSize == that.fetchSize + && Objects.equals(fullDocument, that.fullDocument) + && Objects.equals(fullDocumentBeforeChange, that.fullDocumentBeforeChange); + } + + @Override + public int hashCode() { + return Objects.hash(fetchSize, fullDocument, fullDocumentBeforeChange); + } + + public static MongoChangeStreamOptionsBuilder builder() { + return new MongoChangeStreamOptionsBuilder(); + } + + /** Builder for {@link MongoReadOptions}. */ + @PublicEvolving + public static class MongoChangeStreamOptionsBuilder { + private int fetchSize = CHANGE_STREAM_FETCH_SIZE.defaultValue(); + private FullDocument fullDocument = FullDocument.UPDATE_LOOKUP; + private FullDocumentBeforeChange fullDocumentBeforeChange = FullDocumentBeforeChange.OFF; + + private MongoChangeStreamOptionsBuilder() {} + + /** + * Sets the number of change stream documents should be fetched per round-trip when reading. + * + * @param fetchSize the number of change documents stream should be fetched per round-trip + * when reading. + * @return this builder + */ + public MongoChangeStreamOptionsBuilder setFetchSize(int fetchSize) { + checkArgument(fetchSize > 0, "The change stream fetch size must be larger than 0."); + this.fetchSize = fetchSize; + return this; + } + + /** + * Determines what values your change stream returns on update operations. The default + * setting returns the differences between the original document and the updated document. + * The updateLookup setting returns the differences between the original document and + * updated document as well as a copy of the entire updated document at a point in time + * after the update. The whenAvailable setting returns the updated document, if available. + * The required setting returns the updated document and raises an error if it is not + * available. + * + * @param fullDocument the values your change stream returns on update operations. + * @return this builder + */ + public MongoChangeStreamOptionsBuilder setFullDocument(FullDocument fullDocument) { + this.fullDocument = checkNotNull(fullDocument, "The fullDocument must not be null."); + return this; + } + + /** + * Configures the document pre-image your change stream returns on update operations. The + * pre-image is not available for source records published while copying existing data, and + * the pre-image configuration has no effect on copying. The default setting suppresses the + * document pre-image. The whenAvailable setting returns the document pre-image if it's + * available, before it was replaced, updated, or deleted. The required setting returns the + * document pre-image and raises an error if it is not available. + * + * @param fullDocumentBeforeChange the document pre-image your change stream returns on + * update operations. + * @return this builder + */ + public MongoChangeStreamOptionsBuilder setFullDocumentBeforeChange( + FullDocumentBeforeChange fullDocumentBeforeChange) { + this.fullDocumentBeforeChange = + checkNotNull( + fullDocumentBeforeChange, + "The fullDocumentBeforeChange must not be null."); + return this; + } + + /** + * Build the {@link MongoReadOptions}. + * + * @return a MongoReadOptions with the settings made for this builder. + */ + public MongoChangeStreamOptions build() { + return new MongoChangeStreamOptions(fetchSize, fullDocument, fullDocumentBeforeChange); + } + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java index fb990ed3..b9a14c8c 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java @@ -22,6 +22,9 @@ import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy; import org.apache.flink.connector.mongodb.source.reader.split.MongoScanSourceSplitReader; +import com.mongodb.client.model.changestream.FullDocument; +import com.mongodb.client.model.changestream.FullDocumentBeforeChange; + import java.io.Serializable; import java.util.Objects; @@ -114,6 +117,9 @@ public static class MongoReadOptionsBuilder { private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue(); private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue(); private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue(); + private FullDocument fullDocument = FullDocument.UPDATE_LOOKUP; + private FullDocumentBeforeChange fullDocumentBeforeChange = + FullDocumentBeforeChange.WHEN_AVAILABLE; private MongoReadOptionsBuilder() {} @@ -199,6 +205,44 @@ public MongoReadOptionsBuilder setSamplesPerPartition(int samplesPerPartition) { return this; } + /** + * Determines what values your change stream returns on update operations. The default + * setting returns the differences between the original document and the updated document. + * The updateLookup setting returns the differences between the original document and + * updated document as well as a copy of the entire updated document at a point in time + * after the update. The whenAvailable setting returns the updated document, if available. + * The required setting returns the updated document and raises an error if it is not + * available. + * + * @param fullDocument the values your change stream returns on update operations. + * @return this builder + */ + public MongoReadOptionsBuilder setFullDocument(FullDocument fullDocument) { + this.fullDocument = checkNotNull(fullDocument, "The fullDocument must not be null."); + return this; + } + + /** + * Configures the document pre-image your change stream returns on update operations. The + * pre-image is not available for source records published while copying existing data, and + * the pre-image configuration has no effect on copying. The default setting suppresses the + * document pre-image. The whenAvailable setting returns the document pre-image if it's + * available, before it was replaced, updated, or deleted. The required setting returns the + * document pre-image and raises an error if it is not available. + * + * @param fullDocumentBeforeChange the document pre-image your change stream returns on + * update operations. + * @return this builder + */ + public MongoReadOptionsBuilder setFullDocumentBeforeChange( + FullDocumentBeforeChange fullDocumentBeforeChange) { + this.fullDocumentBeforeChange = + checkNotNull( + fullDocumentBeforeChange, + "The fullDocumentBeforeChange must not be null."); + return this; + } + /** * Build the {@link MongoReadOptions}. * diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoStartupOptions.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoStartupOptions.java new file mode 100644 index 00000000..d33df4d3 --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoStartupOptions.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.connector.mongodb.source.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.source.Boundedness; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The startup options for MongoDB source. */ +@PublicEvolving +public class MongoStartupOptions implements Serializable { + + private static final long serialVersionUID = 1L; + + private final StartupMode startupMode; + @Nullable private final Long startupTimestampMillis; + + private MongoStartupOptions(StartupMode startupMode, @Nullable Long startupTimestampMillis) { + this.startupMode = checkNotNull(startupMode); + this.startupTimestampMillis = startupTimestampMillis; + + switch (startupMode) { + case BOUNDED: + case INITIAL: + case LATEST_OFFSET: + break; + case TIMESTAMP: + checkNotNull(startupTimestampMillis); + break; + default: + throw new UnsupportedOperationException(startupMode + " mode is not supported."); + } + } + + /** The boundedness of startup mode. * */ + public Boundedness boundedness() { + if (startupMode == StartupMode.BOUNDED) { + return Boundedness.BOUNDED; + } else { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + } + + public StartupMode getStartupMode() { + return startupMode; + } + + @Nullable + public Long getStartupTimestampMillis() { + return startupTimestampMillis; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MongoStartupOptions that = (MongoStartupOptions) o; + return Objects.equals(startupMode, that.startupMode) + && Objects.equals(startupTimestampMillis, that.startupTimestampMillis); + } + + @Override + public int hashCode() { + return Objects.hash(startupMode, startupTimestampMillis); + } + + /** Just performs a snapshot on the specified collections upon first startup. */ + public static MongoStartupOptions bounded() { + return new MongoStartupOptions(StartupMode.BOUNDED, null); + } + + /** + * Performs an initial snapshot on the specified collections upon first startup, and continue to + * read the latest change log. + */ + public static MongoStartupOptions initial() { + return new MongoStartupOptions(StartupMode.INITIAL, null); + } + + /** + * Never to perform snapshot on the specified collections upon first startup, just read from the + * end of the change log which means only have the changes since the connector was started. + */ + public static MongoStartupOptions latest() { + return new MongoStartupOptions(StartupMode.LATEST_OFFSET, null); + } + + /** + * Never to perform snapshot on the specified collections upon first startup, and directly read + * change log from the specified timestamp. + * + * @param startupTimestampMillis timestamp for the startup offsets, as milliseconds from epoch. + */ + public static MongoStartupOptions timestamp(long startupTimestampMillis) { + return new MongoStartupOptions(StartupMode.TIMESTAMP, startupTimestampMillis); + } + + /** The startup mode for MongoDB source. */ + @PublicEvolving + public enum StartupMode { + BOUNDED, + INITIAL, + LATEST_OFFSET, + TIMESTAMP + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumState.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumState.java index c81b6354..2312de6f 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumState.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumState.java @@ -19,7 +19,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.connector.mongodb.source.enumerator.assigner.MongoSplitAssigner; -import org.apache.flink.connector.mongodb.source.reader.split.MongoSourceSplitReader; +import org.apache.flink.connector.mongodb.source.reader.split.MongoHybridSourceSplitReader; import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit; import java.util.ArrayList; @@ -47,24 +47,29 @@ public class MongoSourceEnumState { /** * The scan splits that the {@link MongoSourceEnumerator} has assigned to {@link - * MongoSourceSplitReader}s. + * MongoHybridSourceSplitReader}s. */ private final Map assignedScanSplits; - /** The pipeline has been triggered and topic partitions have been assigned to readers. */ + /** The pipeline has been triggered and collection partitions have been assigned to readers. */ private final boolean initialized; + /** Whether the stream split has been assigned. */ + private final boolean streamSplitAssigned; + public MongoSourceEnumState( List remainingCollections, List alreadyProcessedCollections, List remainingScanSplits, Map assignedScanSplits, - boolean initialized) { + boolean initialized, + boolean streamSplitAssigned) { this.remainingCollections = remainingCollections; this.alreadyProcessedCollections = alreadyProcessedCollections; this.remainingScanSplits = remainingScanSplits; this.assignedScanSplits = assignedScanSplits; this.initialized = initialized; + this.streamSplitAssigned = streamSplitAssigned; } public List getRemainingCollections() { @@ -87,9 +92,41 @@ public boolean isInitialized() { return initialized; } + public boolean isStreamSplitAssigned() { + return streamSplitAssigned; + } + /** The initial assignment state for Mongo. */ public static MongoSourceEnumState initialState() { return new MongoSourceEnumState( - new ArrayList<>(), new ArrayList<>(), new ArrayList<>(), new HashMap<>(), false); + new ArrayList<>(), + new ArrayList<>(), + new ArrayList<>(), + new HashMap<>(), + false, + false); + } + + /** The stream enum state for Mongo. */ + public static MongoSourceEnumState streamState(boolean streamSplitAssigned) { + return new MongoSourceEnumState( + new ArrayList<>(), + new ArrayList<>(), + new ArrayList<>(), + new HashMap<>(), + true, + streamSplitAssigned); + } + + /** The hybrid enum state for Mongo. */ + public static MongoSourceEnumState hybridState( + MongoSourceEnumState scanState, boolean streamSplitAssigned) { + return new MongoSourceEnumState( + scanState.remainingCollections, + scanState.alreadyProcessedCollections, + scanState.remainingScanSplits, + scanState.assignedScanSplits, + scanState.initialized, + streamSplitAssigned); } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializer.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializer.java index 32a06faa..eb496b2b 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializer.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializer.java @@ -76,6 +76,8 @@ public byte[] serialize(MongoSourceEnumState state) throws IOException { out.writeBoolean(state.isInitialized()); + out.writeBoolean(state.isStreamSplitAssigned()); + out.flush(); return baos.toByteArray(); } @@ -98,12 +100,25 @@ public MongoSourceEnumState deserialize(int version, byte[] serialized) throws I boolean initialized = in.readBoolean(); + boolean isStreamSplitAssigned; + switch (version) { + case 0: + isStreamSplitAssigned = false; + break; + case 1: + isStreamSplitAssigned = in.readBoolean(); + break; + default: + throw new IOException("Unknown version: " + version); + } + return new MongoSourceEnumState( remainingCollections, alreadyProcessedCollections, remainingScanSplits, assignedScanSplits, - initialized); + initialized, + isStreamSplitAssigned); } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumerator.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumerator.java index 951c5277..6eae5539 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumerator.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumerator.java @@ -47,6 +47,7 @@ public class MongoSourceEnumerator private final SplitEnumeratorContext context; private final MongoSplitAssigner splitAssigner; private final TreeSet readersAwaitingSplit; + private final TreeSet activeReaders; public MongoSourceEnumerator( Boundedness boundedness, @@ -56,6 +57,7 @@ public MongoSourceEnumerator( this.context = context; this.splitAssigner = splitAssigner; this.readersAwaitingSplit = new TreeSet<>(); + this.activeReaders = new TreeSet<>(); } @Override @@ -83,6 +85,7 @@ public void addSplitsBack(List splits, int subtaskId) { @Override public void addReader(int subtaskId) { LOG.debug("Adding reader {} to MongoSourceEnumerator.", subtaskId); + activeReaders.add(subtaskId); } private void assignSplits() { @@ -93,17 +96,22 @@ private void assignSplits() { // if the reader that requested another split has failed in the meantime, remove // it from the list of waiting readers if (!context.registeredReaders().containsKey(nextAwaiting)) { + activeReaders.remove(nextAwaiting); awaitingReader.remove(); continue; } // close idle readers - if (splitAssigner.noMoreSplits() && boundedness == Boundedness.BOUNDED) { - context.signalNoMoreSplits(nextAwaiting); - awaitingReader.remove(); - LOG.info( - "All scan splits have been assigned, closing idle reader {}", nextAwaiting); - continue; + if (splitAssigner.noMoreScanSplits()) { + if (boundedness == Boundedness.BOUNDED || activeReaders.size() > 1) { + context.signalNoMoreSplits(nextAwaiting); + activeReaders.remove(nextAwaiting); + awaitingReader.remove(); + LOG.info( + "All scan splits have been assigned, closing idle reader {}", + nextAwaiting); + continue; + } } Optional split = splitAssigner.getNext(); diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoHybridSplitAssigner.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoHybridSplitAssigner.java new file mode 100644 index 00000000..83aca27d --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoHybridSplitAssigner.java @@ -0,0 +1,135 @@ +/* + * 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.flink.connector.mongodb.source.enumerator.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; +import org.apache.flink.connector.mongodb.source.config.MongoReadOptions; +import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumState; +import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit; +import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit; +import org.apache.flink.connector.mongodb.source.split.MongoStreamOffset; +import org.apache.flink.connector.mongodb.source.split.MongoStreamSourceSplit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.displayCurrentOffset; +import static org.apache.flink.connector.mongodb.source.split.MongoStreamSourceSplit.STREAM_SPLIT_ID; + +/** + * The hybrid split assigner for both {@link MongoScanSourceSplit} and {@link + * MongoStreamSourceSplit}. + */ +@Internal +public class MongoHybridSplitAssigner implements MongoSplitAssigner { + + private static final Logger LOG = LoggerFactory.getLogger(MongoHybridSplitAssigner.class); + + private final MongoConnectionOptions connectionOptions; + private final MongoScanSplitAssigner scanSplitAssigner; + + private boolean isStreamSplitAssigned; + private Map startupOffset; + + public MongoHybridSplitAssigner( + MongoConnectionOptions connectionOptions, + MongoReadOptions readOptions, + MongoSourceEnumState sourceEnumState) { + this.connectionOptions = connectionOptions; + this.isStreamSplitAssigned = sourceEnumState.isStreamSplitAssigned(); + this.scanSplitAssigner = + new MongoScanSplitAssigner(connectionOptions, readOptions, sourceEnumState); + } + + @Override + public Optional getNext() { + if (scanSplitAssigner.noMoreScanSplits()) { + // stream split assigning + if (isStreamSplitAssigned) { + // no more splits for the assigner + return Optional.empty(); + } else { + // assigning the stream split. + isStreamSplitAssigned = true; + MongoStreamSourceSplit streamSplit = + new MongoStreamSourceSplit( + STREAM_SPLIT_ID, + connectionOptions.getDatabase(), + connectionOptions.getCollection(), + MongoStreamOffset.fromOffset(startupOffset)); + LOG.info("Mongo stream split assigned {}", streamSplit); + return Optional.of(streamSplit); + } + } else { + // scan assigner still have remaining splits, assign split from it + return scanSplitAssigner.getNext(); + } + } + + @Override + public void open() { + startupOffset = displayCurrentOffset(connectionOptions).getOffset(); + LOG.info("Initialized startup offset {}", startupOffset); + scanSplitAssigner.open(); + } + + @Override + public void close() throws IOException { + scanSplitAssigner.close(); + } + + @Override + public void addSplitsBack(Collection splits) { + List scanSplits = new ArrayList<>(); + for (MongoSourceSplit split : splits) { + if (split instanceof MongoScanSourceSplit) { + scanSplits.add(split); + } else if (split instanceof MongoStreamSourceSplit) { + MongoStreamSourceSplit streamSplit = (MongoStreamSourceSplit) split; + startupOffset = streamSplit.streamOffset().getOffset(); + isStreamSplitAssigned = false; + } else { + throw new IllegalStateException( + "Unsupported mongo split type " + split.getClass().getName()); + } + } + + if (!scanSplits.isEmpty()) { + scanSplitAssigner.addSplitsBack(scanSplits); + } + } + + @Override + public MongoSourceEnumState snapshotState(long checkpointId) { + return MongoSourceEnumState.hybridState( + scanSplitAssigner.snapshotState(checkpointId), isStreamSplitAssigned); + } + + @Override + public boolean noMoreScanSplits() { + return scanSplitAssigner.noMoreScanSplits(); + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java index e45040d5..1f016968 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java @@ -27,7 +27,6 @@ import com.mongodb.MongoNamespace; import com.mongodb.client.MongoClient; -import com.mongodb.client.MongoClients; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +37,7 @@ import java.util.Map; import java.util.Optional; +import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.clientFor; import static org.apache.flink.util.Preconditions.checkState; /** The split assigner for {@link MongoScanSourceSplit}. */ @@ -53,8 +53,8 @@ public class MongoScanSplitAssigner implements MongoSplitAssigner { private final List alreadyProcessedCollections; private final LinkedList remainingScanSplits; private final Map assignedScanSplits; - private boolean initialized; + private boolean initialized; private MongoClient mongoClient; public MongoScanSplitAssigner( @@ -79,7 +79,7 @@ public void open() { "%s.%s", connectionOptions.getDatabase(), connectionOptions.getCollection()); remainingCollections.add(collectionId); - mongoClient = MongoClients.create(connectionOptions.getUri()); + mongoClient = clientFor(connectionOptions); initialized = true; } } @@ -116,6 +116,9 @@ public void addSplitsBack(Collection splits) { // we should remove the add-backed splits from the assigned list, // because they are failed assignedScanSplits.remove(split.splitId()); + } else { + throw new IllegalArgumentException( + "Cannot add stream split back to scan split assigner."); } } } @@ -127,11 +130,12 @@ public MongoSourceEnumState snapshotState(long checkpointId) { alreadyProcessedCollections, remainingScanSplits, assignedScanSplits, - initialized); + initialized, + false); } @Override - public boolean noMoreSplits() { + public boolean noMoreScanSplits() { checkState(initialized, "The noMoreSplits method was called but not initialized."); return remainingCollections.isEmpty() && remainingScanSplits.isEmpty(); } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java index 97240bb4..a8b9e6bd 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumState; import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumerator; -import org.apache.flink.connector.mongodb.source.reader.split.MongoSourceSplitReader; +import org.apache.flink.connector.mongodb.source.reader.split.MongoHybridSourceSplitReader; import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit; import java.io.IOException; @@ -43,8 +43,8 @@ public interface MongoSplitAssigner { void close() throws IOException; /** - * Gets the next split to assign to {@link MongoSourceSplitReader} when {@link - * MongoSourceEnumerator} receives a split request, until there are {@link #noMoreSplits()}. + * Gets the next split to assign to {@link MongoHybridSourceSplitReader} when {@link + * MongoSourceEnumerator} receives a split request, until there are no more splits. */ Optional getNext(); @@ -57,6 +57,6 @@ public interface MongoSplitAssigner { /** Snapshot the current assign state into checkpoint. */ MongoSourceEnumState snapshotState(long checkpointId); - /** Return whether there are no more splits. */ - boolean noMoreSplits(); + /** Return whether there are no more scan splits. */ + boolean noMoreScanSplits(); } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoStreamSplitAssigner.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoStreamSplitAssigner.java new file mode 100644 index 00000000..2343c51c --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoStreamSplitAssigner.java @@ -0,0 +1,112 @@ +/* + * 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.flink.connector.mongodb.source.enumerator.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; +import org.apache.flink.connector.mongodb.source.config.MongoStartupOptions; +import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumState; +import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit; +import org.apache.flink.connector.mongodb.source.split.MongoStreamOffset; +import org.apache.flink.connector.mongodb.source.split.MongoStreamSourceSplit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.displayCurrentOffset; +import static org.apache.flink.connector.mongodb.source.split.MongoStreamSourceSplit.STREAM_SPLIT_ID; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The split assigner for {@link MongoStreamSourceSplit}. */ +@Internal +public class MongoStreamSplitAssigner implements MongoSplitAssigner { + + private static final Logger LOG = LoggerFactory.getLogger(MongoStreamSplitAssigner.class); + + private final MongoConnectionOptions connectionOptions; + private final MongoStartupOptions startupOptions; + + private boolean isStreamSplitAssigned; + private Map startupOffset; + + public MongoStreamSplitAssigner( + MongoConnectionOptions connectionOptions, MongoStartupOptions startupOptions) { + this.connectionOptions = connectionOptions; + this.startupOptions = startupOptions; + } + + @Override + public void open() { + switch (startupOptions.getStartupMode()) { + case LATEST_OFFSET: + this.startupOffset = displayCurrentOffset(connectionOptions).getOffset(); + break; + case TIMESTAMP: + this.startupOffset = + MongoStreamOffset.fromTimeMillis( + checkNotNull(startupOptions.getStartupTimestampMillis())) + .getOffset(); + break; + default: + throw new IllegalStateException( + "Unsupported startup mode " + startupOptions.getStartupMode()); + } + } + + @Override + public void close() throws IOException {} + + @Override + public Optional getNext() { + if (isStreamSplitAssigned) { + return Optional.empty(); + } else { + isStreamSplitAssigned = true; + MongoStreamSourceSplit streamSplit = + new MongoStreamSourceSplit( + STREAM_SPLIT_ID, + connectionOptions.getDatabase(), + connectionOptions.getCollection(), + MongoStreamOffset.fromOffset(startupOffset)); + + LOG.info("Mongo stream split assigned {}", streamSplit); + return Optional.of(streamSplit); + } + } + + @Override + public void addSplitsBack(Collection splits) { + // will re-create stream split later + isStreamSplitAssigned = false; + } + + @Override + public MongoSourceEnumState snapshotState(long checkpointId) { + return MongoSourceEnumState.streamState(isStreamSplitAssigned); + } + + @Override + public boolean noMoreScanSplits() { + return true; + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceReader.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceReader.java index 5eb6669d..a4bfd975 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceReader.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceReader.java @@ -28,14 +28,17 @@ import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplitState; import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit; import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitState; +import org.apache.flink.connector.mongodb.source.split.MongoStreamSourceSplit; +import org.apache.flink.connector.mongodb.source.split.MongoStreamSourceSplitState; -import org.bson.BsonDocument; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Map; import java.util.function.Supplier; +import static org.apache.flink.util.Preconditions.checkState; + /** * The common mongo source reader for both ordered & unordered message consuming. * @@ -44,14 +47,14 @@ @Internal public class MongoSourceReader extends SingleThreadMultiplexSourceReaderBase< - BsonDocument, OUT, MongoSourceSplit, MongoSourceSplitState> { + MongoSourceRecord, OUT, MongoSourceSplit, MongoSourceSplitState> { private static final Logger LOG = LoggerFactory.getLogger(MongoSourceReader.class); public MongoSourceReader( - FutureCompletingBlockingQueue> elementQueue, - Supplier> splitReaderSupplier, - RecordEmitter recordEmitter, + FutureCompletingBlockingQueue> elementQueue, + Supplier> splitReaderSupplier, + RecordEmitter recordEmitter, MongoSourceReaderContext readerContext) { super( elementQueue, @@ -72,8 +75,14 @@ public void start() { protected void onSplitFinished(Map finishedSplitIds) { for (MongoSourceSplitState splitState : finishedSplitIds.values()) { MongoSourceSplit sourceSplit = splitState.toMongoSourceSplit(); + checkState( + sourceSplit instanceof MongoScanSourceSplit, + String.format( + "Only scan split could finish, but the actual split is stream split %s", + sourceSplit)); LOG.info("Split {} is finished.", sourceSplit.splitId()); } + context.sendSplitRequest(); } @@ -81,8 +90,10 @@ protected void onSplitFinished(Map finishedSplitI protected MongoSourceSplitState initializedState(MongoSourceSplit split) { if (split instanceof MongoScanSourceSplit) { return new MongoScanSourceSplitState((MongoScanSourceSplit) split); + } else if (split instanceof MongoStreamSourceSplit) { + return new MongoStreamSourceSplitState((MongoStreamSourceSplit) split); } else { - throw new IllegalArgumentException("Unknown split type."); + throw new IllegalStateException("Unknown split type " + split.getClass().getName()); } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceRecord.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceRecord.java new file mode 100644 index 00000000..025c0cd5 --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceRecord.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * http://www.apache.org/licenses/LICENSE-2.0 + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.flink.connector.mongodb.source.reader; + +import org.apache.flink.annotation.PublicEvolving; + +import org.bson.BsonDocument; +import org.bson.BsonString; + +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.DEFAULT_JSON_WRITER_SETTINGS; + +/** Source record produced by {@link MongoSourceReader}. * */ +@PublicEvolving +public class MongoSourceRecord { + + public static final String TYPE_FIELD = "type"; + public static final String RECORD_FIELD = "record"; + + private final RecordType type; + private final BsonDocument record; + + private MongoSourceRecord(RecordType type, BsonDocument record) { + this.type = type; + this.record = record; + } + + public static MongoSourceRecord snapshotRecord(BsonDocument record) { + return new MongoSourceRecord(RecordType.SNAPSHOT, record); + } + + public static MongoSourceRecord streamRecord(BsonDocument record) { + return new MongoSourceRecord(RecordType.STREAM, record); + } + + public static MongoSourceRecord heartbeatRecord(BsonDocument record) { + return new MongoSourceRecord(RecordType.HEARTBEAT, record); + } + + public RecordType getType() { + return type; + } + + public BsonDocument getRecord() { + return record; + } + + public String toJson() { + return new BsonDocument(TYPE_FIELD, new BsonString(type.name())) + .append(RECORD_FIELD, record) + .toJson(DEFAULT_JSON_WRITER_SETTINGS); + } + + /** Enum to describe record type for {@link MongoSourceRecord}. */ + @PublicEvolving + public enum RecordType { + SNAPSHOT, + STREAM, + HEARTBEAT + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoDeserializationSchema.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoDeserializationSchema.java index 1a138f70..02507b29 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoDeserializationSchema.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoDeserializationSchema.java @@ -19,16 +19,13 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; import org.apache.flink.util.Collector; -import org.bson.BsonDocument; - -import java.io.IOException; import java.io.Serializable; /** - * A schema bridge for deserializing the MongoDB's {@code BsonDocument} into a flink managed - * instance. + * A schema bridge for deserializing the {@code MongoSourceRecord} into a flink managed instance. * * @param The output message type for sinking to downstream flink operator. */ @@ -36,28 +33,15 @@ public interface MongoDeserializationSchema extends Serializable, ResultTypeQueryable { /** - * Deserializes the BSON document. - * - * @param document The BSON document to deserialize. - * @return The deserialized message as an object (null if the message cannot be deserialized). - */ - T deserialize(BsonDocument document) throws IOException; - - /** - * Deserializes the BSON document. + * Deserializes the Mongo source record. * *

Can output multiple records through the {@link Collector}. Note that number and size of * the produced records should be relatively small. Depending on the source implementation * records can be buffered in memory or collecting records might delay emitting checkpoint * barrier. * - * @param document The BSON document to deserialize. + * @param sourceRecord The source record to deserialize. * @param out The collector to put the resulting messages. */ - default void deserialize(BsonDocument document, Collector out) throws IOException { - T deserialize = deserialize(document); - if (deserialize != null) { - out.collect(deserialize); - } - } + void deserialize(MongoSourceRecord sourceRecord, Collector out); } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/emitter/MongoRecordEmitter.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/emitter/MongoRecordEmitter.java index ab0d6aae..efda7341 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/emitter/MongoRecordEmitter.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/emitter/MongoRecordEmitter.java @@ -21,19 +21,18 @@ import org.apache.flink.api.connector.source.SourceOutput; import org.apache.flink.connector.base.source.reader.RecordEmitter; import org.apache.flink.connector.mongodb.source.reader.MongoSourceReader; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema; import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitState; import org.apache.flink.util.Collector; -import org.bson.BsonDocument; - /** * The {@link RecordEmitter} implementation for {@link MongoSourceReader} . We would always update * the last consumed message id in this emitter. */ @Internal public class MongoRecordEmitter - implements RecordEmitter { + implements RecordEmitter { private final MongoDeserializationSchema deserializationSchema; private final SourceOutputWrapper sourceOutputWrapper; @@ -45,13 +44,27 @@ public MongoRecordEmitter(MongoDeserializationSchema deserializationSchema) { @Override public void emitRecord( - BsonDocument document, SourceOutput output, MongoSourceSplitState splitState) + MongoSourceRecord sourceRecord, + SourceOutput output, + MongoSourceSplitState splitState) throws Exception { - // Update current offset. - splitState.updateOffset(document); - // Sink the record to source output. - sourceOutputWrapper.setSourceOutput(output); - deserializationSchema.deserialize(document, sourceOutputWrapper); + switch (sourceRecord.getType()) { + case HEARTBEAT: + // Update current offset from heartbeat. + splitState.updateOffset(sourceRecord); + break; + case STREAM: + case SNAPSHOT: + // Sink the record to source output. + sourceOutputWrapper.setSourceOutput(output); + deserializationSchema.deserialize(sourceRecord, sourceOutputWrapper); + // Update current offset from record. + splitState.updateOffset(sourceRecord); + break; + default: + throw new IllegalStateException( + "Unsupported record type " + sourceRecord.getType()); + } } private static class SourceOutputWrapper implements Collector { diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoHybridSourceSplitReader.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoHybridSourceSplitReader.java new file mode 100644 index 00000000..9a305ef2 --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoHybridSourceSplitReader.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.flink.connector.mongodb.source.reader.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; +import org.apache.flink.connector.mongodb.source.config.MongoChangeStreamOptions; +import org.apache.flink.connector.mongodb.source.config.MongoReadOptions; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceReaderContext; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; +import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit; +import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.List; + +/** A split reader implements {@link SplitReader} for {@link MongoSourceSplit}. */ +@Internal +public class MongoHybridSourceSplitReader implements MongoSourceSplitReader { + + private static final Logger LOG = LoggerFactory.getLogger(MongoHybridSourceSplitReader.class); + + private final MongoConnectionOptions connectionOptions; + private final MongoReadOptions readOptions; + private final MongoChangeStreamOptions changeStreamOptions; + private final MongoSourceReaderContext readerContext; + @Nullable private final List projectedFields; + @Nullable private MongoSourceSplitReader currentReader; + + public MongoHybridSourceSplitReader( + MongoConnectionOptions connectionOptions, + MongoReadOptions readOptions, + MongoChangeStreamOptions changeStreamOptions, + MongoSourceReaderContext readerContext, + @Nullable List projectedFields) { + this.connectionOptions = connectionOptions; + this.readOptions = readOptions; + this.changeStreamOptions = changeStreamOptions; + this.readerContext = readerContext; + this.projectedFields = projectedFields; + } + + @Override + public RecordsWithSplitIds fetch() throws IOException { + if (currentReader == null) { + throw new IOException("Split reader is not ready."); + } + return currentReader.fetch(); + } + + @Override + public void handleSplitsChanges(SplitsChange splitsChanges) { + if (!(splitsChanges instanceof SplitsAddition)) { + throw new UnsupportedOperationException( + String.format( + "The SplitChange type of %s is not supported.", + splitsChanges.getClass())); + } + + switchReaderWhenNeeded(splitsChanges); + } + + private void switchReaderWhenNeeded(SplitsChange splitsChanges) { + // the stream reader should keep alive + if (currentReader instanceof MongoStreamSourceSplitReader) { + return; + } + + if (canAcceptNextSplit()) { + MongoSourceSplit sourceSplit = splitsChanges.splits().get(0); + if (sourceSplit instanceof MongoScanSourceSplit) { + if (currentReader == null) { + currentReader = + new MongoScanSourceSplitReader( + connectionOptions, readOptions, projectedFields, readerContext); + } + } else { + // switch to stream reader + if (currentReader != null) { + try { + currentReader.close(); + } catch (Exception e) { + // ignore the exception + LOG.warn("Cannot close reader gracefully", e); + } + } + currentReader = + new MongoStreamSourceSplitReader( + connectionOptions, changeStreamOptions, readerContext); + } + currentReader.handleSplitsChanges(splitsChanges); + } + } + + @Override + public void wakeUp() { + if (currentReader != null) { + LOG.info("Wakeup current reader {}", currentReader.getClass().getName()); + currentReader.wakeUp(); + } + } + + @Override + public void close() throws Exception { + if (currentReader != null) { + LOG.info("Close current reader {}", currentReader.getClass().getName()); + currentReader.close(); + currentReader = null; + } + } + + @Override + public boolean canAcceptNextSplit() { + return currentReader == null || currentReader.canAcceptNextSplit(); + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoScanSourceSplitReader.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoScanSourceSplitReader.java index 134fe73a..86834a4b 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoScanSourceSplitReader.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoScanSourceSplitReader.java @@ -26,14 +26,15 @@ import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; import org.apache.flink.connector.mongodb.source.config.MongoReadOptions; import org.apache.flink.connector.mongodb.source.reader.MongoSourceReaderContext; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit; import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit; import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.IOUtils; import com.mongodb.MongoException; import com.mongodb.client.FindIterable; import com.mongodb.client.MongoClient; -import com.mongodb.client.MongoClients; import com.mongodb.client.MongoCursor; import org.bson.BsonDocument; import org.slf4j.Logger; @@ -44,11 +45,13 @@ import java.io.IOException; import java.util.List; +import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.clientFor; import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.project; +import static org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord.snapshotRecord; -/** An split reader implements {@link SplitReader} for {@link MongoScanSourceSplit}. */ +/** A split reader implements {@link SplitReader} for {@link MongoScanSourceSplit}. */ @Internal -public class MongoScanSourceSplitReader implements MongoSourceSplitReader { +public class MongoScanSourceSplitReader implements MongoSourceSplitReader { private static final Logger LOG = LoggerFactory.getLogger(MongoScanSourceSplitReader.class); @@ -75,12 +78,12 @@ public MongoScanSourceSplitReader( } @Override - public RecordsWithSplitIds fetch() throws IOException { + public RecordsWithSplitIds fetch() throws IOException { if (closed) { - throw new IllegalStateException("Cannot fetch records from a closed split reader"); + throw new IOException("Cannot fetch records from a closed split reader"); } - RecordsBySplits.Builder builder = new RecordsBySplits.Builder<>(); + RecordsBySplits.Builder builder = new RecordsBySplits.Builder<>(); // Return when no split registered to this reader. if (currentSplit == null) { @@ -95,13 +98,12 @@ public RecordsWithSplitIds fetch() throws IOException { return builder.build(); } - currentCursor = getOrCreateCursor(); int fetchSize = readOptions.getFetchSize(); - try { + currentCursor = getOrCreateCursor(); for (int recordNum = 0; recordNum < fetchSize; recordNum++) { if (currentCursor.hasNext()) { - builder.add(currentSplit, currentCursor.next()); + builder.add(currentSplit, snapshotRecord(currentCursor.next())); readerContext.getReadCount().incrementAndGet(); if (readerContext.isOverLimit()) { builder.addFinishedSplit(currentSplit.splitId()); @@ -143,8 +145,10 @@ public void handleSplitsChanges(SplitsChange splitsChanges) { sourceSplit.getClass())); } - this.currentSplit = (MongoScanSourceSplit) sourceSplit; - this.finished = false; + if (canAcceptNextSplit()) { + this.currentSplit = (MongoScanSourceSplit) sourceSplit; + this.finished = false; + } } @Override @@ -164,7 +168,7 @@ public void close() { private MongoCursor getOrCreateCursor() { if (currentCursor == null) { LOG.debug("Opened cursor for partitionId: {}", currentSplit); - mongoClient = MongoClients.create(connectionOptions.getUri()); + mongoClient = clientFor(connectionOptions); // Using MongoDB's cursor.min() and cursor.max() to limit an index bound. // When the index range is the primary key, the bound is (min <= _id < max). @@ -201,18 +205,14 @@ private MongoCursor getOrCreateCursor() { } private void closeCursor() { - if (currentCursor != null) { - LOG.debug("Closing cursor for split: {}", currentSplit); - try { - currentCursor.close(); - } finally { - currentCursor = null; - try { - mongoClient.close(); - } finally { - mongoClient = null; - } - } - } + LOG.debug("Closing scan cursor for split: {}", currentSplit); + IOUtils.closeAllQuietly(currentCursor, mongoClient); + currentCursor = null; + mongoClient = null; + } + + @Override + public boolean canAcceptNextSplit() { + return currentSplit == null || finished; } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoSourceSplitReader.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoSourceSplitReader.java index 1a88adf3..e36ffbc2 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoSourceSplitReader.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoSourceSplitReader.java @@ -19,15 +19,12 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit; -import org.bson.BsonDocument; - -/** - * An split reader implements {@link SplitReader} for {@link MongoSourceSplit}. - * - * @param Mongo source split. - */ +/** A split reader implements {@link SplitReader} for {@link MongoSourceSplit}. */ @Internal -public interface MongoSourceSplitReader - extends SplitReader {} +public interface MongoSourceSplitReader extends SplitReader { + /** Whether the current reader can accept the next split. */ + boolean canAcceptNextSplit(); +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoStreamSourceSplitReader.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoStreamSourceSplitReader.java new file mode 100644 index 00000000..2bf69e75 --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoStreamSourceSplitReader.java @@ -0,0 +1,254 @@ +/* + * 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.flink.connector.mongodb.source.reader.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; +import org.apache.flink.connector.mongodb.source.config.MongoChangeStreamOptions; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceReaderContext; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; +import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit; +import org.apache.flink.connector.mongodb.source.split.MongoStreamSourceSplit; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.IOUtils; + +import com.mongodb.MongoCommandException; +import com.mongodb.client.ChangeStreamIterable; +import com.mongodb.client.MongoChangeStreamCursor; +import com.mongodb.client.MongoClient; +import org.bson.BsonDocument; +import org.bson.BsonTimestamp; +import org.bson.RawBsonDocument; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.NoSuchElementException; +import java.util.Optional; + +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.CLUSTER_TIME_FIELD; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.FAILED_TO_PARSE_ERROR; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ILLEGAL_OPERATION_ERROR; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.RESUME_TOKEN_FIELD; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UNAUTHORIZED_ERROR; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UNKNOWN_FIELD_ERROR; +import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.clientFor; +import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.currentClusterTime; +import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.getChangeStreamIterable; +import static org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord.heartbeatRecord; +import static org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord.streamRecord; + +/** A split reader implements {@link SplitReader} for {@link MongoStreamSourceSplit}. */ +@Internal +public class MongoStreamSourceSplitReader implements MongoSourceSplitReader { + + private static final Logger LOG = LoggerFactory.getLogger(MongoScanSourceSplitReader.class); + + private final MongoConnectionOptions connectionOptions; + private final MongoChangeStreamOptions changeStreamOptions; + private final MongoSourceReaderContext readerContext; + + private boolean closed = false; + private MongoClient mongoClient; + private MongoChangeStreamCursor currentCursor; + private MongoStreamSourceSplit currentSplit; + + private boolean supportsStartAtOperationTime = true; + private boolean supportsStartAfter = true; + private boolean supportsFullDocumentBeforeChange = true; + + public MongoStreamSourceSplitReader( + MongoConnectionOptions connectionOptions, + MongoChangeStreamOptions changeStreamOptions, + MongoSourceReaderContext readerContext) { + this.connectionOptions = connectionOptions; + this.changeStreamOptions = changeStreamOptions; + this.readerContext = readerContext; + } + + @Override + public RecordsWithSplitIds fetch() throws IOException { + if (closed) { + throw new IOException("Cannot fetch records from a closed split reader"); + } + + RecordsBySplits.Builder builder = new RecordsBySplits.Builder<>(); + + // Return when no split registered to this reader. + if (currentSplit == null) { + return builder.build(); + } + + if (currentCursor == null) { + currentCursor = openChangeStreamCursor(); + } + + try { + for (int recordNum = 0; recordNum < changeStreamOptions.getFetchSize(); recordNum++) { + BsonDocument nextRecord = currentCursor.tryNext(); + if (nextRecord != null) { + builder.add(currentSplit, streamRecord(nextRecord)); + } else { + builder.add(currentSplit, heartbeatRecord(heartbeat())); + break; + } + } + return builder.build(); + } catch (Exception e) { + throw new IOException("Poll change stream records failed", e); + } + } + + @Override + public void handleSplitsChanges(SplitsChange splitsChanges) { + LOG.debug("Handle split changes {}", splitsChanges); + + if (!(splitsChanges instanceof SplitsAddition)) { + throw new UnsupportedOperationException( + String.format( + "The SplitChange type of %s is not supported.", + splitsChanges.getClass().getName())); + } + + MongoSourceSplit sourceSplit = splitsChanges.splits().get(0); + if (!(sourceSplit instanceof MongoStreamSourceSplit)) { + throw new UnsupportedOperationException( + String.format( + "The SourceSplit type of %s is not supported.", + sourceSplit.getClass().getName())); + } + + if (canAcceptNextSplit()) { + this.currentSplit = (MongoStreamSourceSplit) sourceSplit; + } + } + + @Override + public void wakeUp() { + // Close current cursor to cancel blocked hasNext(), next(). + closeCursor(); + } + + @Override + public void close() throws Exception { + if (!closed) { + closed = true; + closeCursor(); + } + } + + private MongoChangeStreamCursor openChangeStreamCursor() { + LOG.debug("Opened change stream cursor for partitionId: {}", currentSplit); + mongoClient = clientFor(connectionOptions); + + ChangeStreamIterable changeStreamIterable = + getChangeStreamIterable( + mongoClient, currentSplit.getDatabase(), currentSplit.getCollection()); + + Optional resumeToken = currentSplit.streamOffset().getResumeToken(); + Optional operationTime = currentSplit.streamOffset().getClusterTime(); + + if (resumeToken.isPresent()) { + if (supportsStartAfter) { + LOG.info("Open the change stream after the previous offset: {}", resumeToken.get()); + changeStreamIterable.startAfter(resumeToken.get()); + } else { + LOG.info( + "Open the change stream after the previous offset using resumeAfter: {}", + resumeToken.get()); + changeStreamIterable.resumeAfter(resumeToken.get()); + } + } else { + if (supportsStartAtOperationTime) { + changeStreamIterable.startAtOperationTime( + operationTime.orElseThrow(NoSuchElementException::new)); + LOG.info("Open the change stream at the operationTime: {}", operationTime.get()); + } else { + LOG.warn("Open the change stream of the latest offset"); + } + } + + changeStreamIterable.fullDocument(changeStreamOptions.getFullDocument()); + if (supportsFullDocumentBeforeChange) { + changeStreamIterable.fullDocumentBeforeChange( + changeStreamOptions.getFullDocumentBeforeChange()); + } + + try { + return (MongoChangeStreamCursor) + changeStreamIterable.withDocumentClass(RawBsonDocument.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(); + } else if (e.getErrorMessage().contains("startAfter")) { + supportsStartAfter = false; + return openChangeStreamCursor(); + } else if (e.getErrorMessage().contains("fullDocumentBeforeChange")) { + supportsFullDocumentBeforeChange = false; + return openChangeStreamCursor(); + } else { + LOG.error("Open change stream failed ", e); + throw new FlinkRuntimeException("Open change stream failed", e); + } + } else if (e.getErrorCode() == ILLEGAL_OPERATION_ERROR) { + LOG.error( + "Illegal $changeStream operation: {} {}", + e.getErrorMessage(), + e.getErrorCode()); + throw new FlinkRuntimeException("Illegal $changeStream operation", e); + } else if (e.getErrorCode() == UNAUTHORIZED_ERROR) { + LOG.error( + "Unauthorized $changeStream operation: {} {}", + e.getErrorMessage(), + e.getErrorCode()); + throw new FlinkRuntimeException("Unauthorized $changeStream operation", e); + } else { + LOG.error("Open change stream failed ", e); + throw new FlinkRuntimeException("Open change stream failed", e); + } + } + } + + private void closeCursor() { + LOG.debug("Closing change stream cursor for split: {}", currentSplit); + IOUtils.closeAllQuietly(currentCursor, mongoClient); + currentCursor = null; + mongoClient = null; + } + + private BsonDocument heartbeat() { + if (currentCursor.getResumeToken() != null) { + return new BsonDocument(RESUME_TOKEN_FIELD, currentCursor.getResumeToken()); + } else { + return new BsonDocument(CLUSTER_TIME_FIELD, currentClusterTime(mongoClient)); + } + } + + @Override + public boolean canAcceptNextSplit() { + return currentSplit == null; + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplit.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplit.java index c7560742..e5159751 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplit.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplit.java @@ -117,4 +117,24 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(super.hashCode(), database, collection, min, max, hint, offset); } + + @Override + public String toString() { + return "MongoScanSourceSplit {" + + " splitId=" + + splitId + + ", database=" + + database + + ", collection=" + + collection + + ", min=" + + min + + ", max=" + + max + + ", hint=" + + hint + + ", offset=" + + offset + + " }"; + } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplitState.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplitState.java index 7b3b2abb..430f55f8 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplitState.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoScanSourceSplitState.java @@ -18,8 +18,7 @@ package org.apache.flink.connector.mongodb.source.split; import org.apache.flink.annotation.Internal; - -import org.bson.BsonDocument; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; /** MongoDB source split state for {@link MongoScanSourceSplit}. */ @Internal @@ -47,7 +46,7 @@ public MongoScanSourceSplit toMongoSourceSplit() { } @Override - public void updateOffset(BsonDocument record) { + public void updateOffset(MongoSourceRecord record) { offset++; } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitSerializer.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitSerializer.java index c5ab7fd1..5b5bfbc3 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitSerializer.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitSerializer.java @@ -24,9 +24,14 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; +import java.io.DataInput; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; +import java.util.Map; + +import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeMap; +import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeMap; /** The {@link SimpleVersionedSerializer serializer} for {@link MongoSourceSplit}. */ @Internal @@ -35,9 +40,10 @@ public class MongoSourceSplitSerializer implements SimpleVersionedSerializer offset = + deserializeMap(in, DataInput::readUTF, DataInput::readUTF); + MongoStreamOffset streamOffset = MongoStreamOffset.fromOffset(offset); + return new MongoStreamSourceSplit(splitId, database, collection, streamOffset); + default: + throw new IOException("Unknown version: " + version); + } + } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitState.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitState.java index 782c9134..b657b1a0 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitState.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitState.java @@ -18,8 +18,7 @@ package org.apache.flink.connector.mongodb.source.split; import org.apache.flink.annotation.Internal; - -import org.bson.BsonDocument; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; /** MongoDB source split state for {@link MongoSourceSplit}. */ @Internal @@ -33,5 +32,5 @@ public interface MongoSourceSplitState { * * @param record The latest record that was read. */ - void updateOffset(BsonDocument record); + void updateOffset(MongoSourceRecord record); } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamOffset.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamOffset.java new file mode 100644 index 00000000..dc13d4e7 --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamOffset.java @@ -0,0 +1,118 @@ +/* + * 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.flink.connector.mongodb.source.split; + +import org.apache.flink.annotation.PublicEvolving; + +import org.bson.BsonDocument; +import org.bson.BsonTimestamp; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.TimeUnit; + +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.CLUSTER_TIME_FIELD; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.DEFAULT_JSON_WRITER_SETTINGS; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.RESUME_TOKEN_FIELD; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The MongoDB change stream offset, record resumeToken and clusterTime. */ +@PublicEvolving +public class MongoStreamOffset { + + private Map offset; + + private MongoStreamOffset(BsonDocument resumeToken) { + Map offset = new HashMap<>(); + offset.put(RESUME_TOKEN_FIELD, resumeToken.toJson(DEFAULT_JSON_WRITER_SETTINGS)); + this.offset = offset; + } + + private MongoStreamOffset(BsonTimestamp clusterTime) { + Map offset = new HashMap<>(); + offset.put(CLUSTER_TIME_FIELD, String.valueOf(clusterTime.getValue())); + this.offset = offset; + } + + private MongoStreamOffset(Map offset) { + this.offset = checkNotNull(offset); + } + + public static MongoStreamOffset fromResumeToken(BsonDocument resumeToken) { + return new MongoStreamOffset(resumeToken); + } + + public static MongoStreamOffset fromClusterTime(BsonTimestamp clusterTime) { + return new MongoStreamOffset(clusterTime); + } + + public static MongoStreamOffset fromHeartbeat(BsonDocument heartbeat) { + if (heartbeat.containsKey(RESUME_TOKEN_FIELD)) { + return fromResumeToken(heartbeat.getDocument(RESUME_TOKEN_FIELD)); + } else { + return fromClusterTime(heartbeat.getTimestamp(CLUSTER_TIME_FIELD)); + } + } + + public static MongoStreamOffset fromTimeMillis(long timeMillis) { + int seconds = (int) TimeUnit.MILLISECONDS.toSeconds(timeMillis); + return new MongoStreamOffset(new BsonTimestamp(seconds, 0)); + } + + public static MongoStreamOffset fromOffset(Map offset) { + return new MongoStreamOffset(offset); + } + + public Optional getResumeToken() { + return Optional.ofNullable(offset.get(RESUME_TOKEN_FIELD)).map(BsonDocument::parse); + } + + public Optional getClusterTime() { + return Optional.ofNullable(offset.get(CLUSTER_TIME_FIELD)) + .map(Long::valueOf) + .map(BsonTimestamp::new); + } + + public Map getOffset() { + return offset; + } + + @Override + public String toString() { + return offset.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MongoStreamOffset that = (MongoStreamOffset) o; + return Objects.equals(offset, that.offset); + } + + @Override + public int hashCode() { + return Objects.hash(offset); + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamSourceSplit.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamSourceSplit.java new file mode 100644 index 00000000..1549dc45 --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamSourceSplit.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.flink.connector.mongodb.source.split; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.connector.source.SourceSplit; + +import java.util.Objects; + +/** A {@link SourceSplit} implementation for a MongoDB's change streams. */ +@PublicEvolving +public class MongoStreamSourceSplit extends MongoSourceSplit { + + private static final long serialVersionUID = 1L; + + public static final String STREAM_SPLIT_ID = "stream-split"; + + private final String database; + + private final String collection; + + private final MongoStreamOffset offset; + + public MongoStreamSourceSplit( + String splitId, String database, String collection, MongoStreamOffset offset) { + super(splitId); + this.database = database; + this.collection = collection; + this.offset = offset; + } + + public String getDatabase() { + return database; + } + + public String getCollection() { + return collection; + } + + public MongoStreamOffset streamOffset() { + return offset; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + MongoStreamSourceSplit split = (MongoStreamSourceSplit) o; + return Objects.equals(database, split.database) + && Objects.equals(collection, split.collection) + && Objects.equals(offset, split.offset); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), database, collection, offset); + } + + @Override + public String toString() { + return "MongoStreamSourceSplit {" + + " splitId=" + + splitId + + ", database=" + + database + + ", collection=" + + collection + + ", offset=" + + offset + + " }"; + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamSourceSplitState.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamSourceSplitState.java new file mode 100644 index 00000000..82d1ce32 --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoStreamSourceSplitState.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.flink.connector.mongodb.source.split; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; + +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD; + +/** MongoDB source split state for {@link MongoStreamSourceSplit}. */ +@Internal +public class MongoStreamSourceSplitState implements MongoSourceSplitState { + + private final MongoStreamSourceSplit streamSplit; + + private MongoStreamOffset offset; + + public MongoStreamSourceSplitState(MongoStreamSourceSplit streamSplit) { + this.streamSplit = streamSplit; + this.offset = streamSplit.streamOffset(); + } + + @Override + public MongoStreamSourceSplit toMongoSourceSplit() { + return new MongoStreamSourceSplit( + streamSplit.splitId(), + streamSplit.getDatabase(), + streamSplit.getCollection(), + offset); + } + + @Override + public void updateOffset(MongoSourceRecord record) { + switch (record.getType()) { + case HEARTBEAT: + this.offset = MongoStreamOffset.fromHeartbeat(record.getRecord()); + break; + case STREAM: + this.offset = + MongoStreamOffset.fromResumeToken(record.getRecord().getDocument(ID_FIELD)); + break; + default: + throw new IllegalArgumentException( + "Expected stream or heartbeat record, but " + record.getType()); + } + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java index a91c4892..c6f4e31c 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java @@ -20,12 +20,17 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.DescribedEnum; import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.description.InlineElement; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy; +import org.apache.flink.connector.mongodb.table.config.FullDocumentStrategy; import java.time.Duration; +import static org.apache.flink.configuration.description.TextElement.text; + /** * Base options for the MongoDB connector. Needs to be public so that the {@link * org.apache.flink.table.api.TableDescriptor} can access it. @@ -99,6 +104,23 @@ private MongoConnectorOptions() {} + "Then uses every 'scan.partition.samples' as the value to use to calculate the partition boundaries." + "The total number of samples taken is calculated as: samples per partition * (count of documents / number of documents per partition."); + public static final ConfigOption SCAN_STARTUP_MODE = + ConfigOptions.key("scan.startup.mode") + .enumType(ScanStartupMode.class) + .defaultValue(ScanStartupMode.BOUNDED) + .withDescription( + "Startup mode for MongoDB connector. valid enumerations are 'bounded', 'initial', 'latest-offset', and 'timestamp'. " + + "bounded: bounded read collection snapshot data. " + + "initial: read collection snapshot data and then continuously read changed data. " + + "latest-offset: continuously read changed data from latest offset of oplog. " + + "timestamp: continuously read changed data from specified timestamp offset of oplog."); + + public static final ConfigOption SCAN_STARTUP_TIMESTAMP_MILLIS = + ConfigOptions.key("scan.startup.timestamp-millis") + .longType() + .noDefaultValue() + .withDescription("Optional timestamp used in case of 'timestamp' startup mode"); + public static final ConfigOption LOOKUP_RETRY_INTERVAL = ConfigOptions.key("lookup.retry.interval") .durationType() @@ -106,6 +128,22 @@ private MongoConnectorOptions() {} .withDescription( "Specifies the retry time interval if lookup records from database failed."); + public static final ConfigOption CHANGE_STREAM_FETCH_SIZE = + ConfigOptions.key("change-stream.fetch-size") + .intType() + .defaultValue(2048) + .withDescription( + "Gives the reader a hint as to the number of change stream documents that should be fetched from the database per round-trip when reading. "); + + public static final ConfigOption CHANGE_STREAM_FULL_DOCUMENT_STRATEGY = + ConfigOptions.key("change-stream.full-document.strategy") + .enumType(FullDocumentStrategy.class) + .defaultValue(FullDocumentStrategy.UPDATE_LOOKUP) + .withDescription( + "Specifies the full document strategy. Available strategies are update-lookup, and pre-and-post-images." + + "update-lookup: prior to version 6.0 of MongoDB, the pre images was not stored in the oplog. To obtain complete updated row data, use updateLookup feature to lookup the latest snapshot when the oplog record is accessed." + + "pre-and-post-images: starting in MongoDB 6.0, you can use change stream events to output the version of a document before and after changes (the document pre- and post-images)."); + public static final ConfigOption BUFFER_FLUSH_MAX_ROWS = ConfigOptions.key("sink.buffer-flush.max-rows") .intType() @@ -139,4 +177,34 @@ private MongoConnectorOptions() {} .defaultValue(Duration.ofMillis(1000L)) .withDescription( "Specifies the retry time interval if writing records to database failed."); + + /** Startup mode for the MongoDB connector, see {@link #SCAN_STARTUP_MODE}. */ + public enum ScanStartupMode implements DescribedEnum { + BOUNDED("bounded", text("Bounded read collection snapshot.")), + INITIAL("initial", text("Read collection snapshot and continuously read changed data.")), + LATEST_OFFSET( + "latest-offset", + text("Continuously read changed data from latest offset of oplog.")), + TIMESTAMP( + "timestamp", + text("Continuously read changed data from specified timestamp offset of oplog.")); + + private final String value; + private final InlineElement description; + + ScanStartupMode(String value, InlineElement description) { + this.value = value; + this.description = description; + } + + @Override + public String toString() { + return value; + } + + @Override + public InlineElement getDescription() { + return description; + } + } } diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactory.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactory.java index 1430f23a..343f1a3e 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactory.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions; +import org.apache.flink.connector.mongodb.source.config.MongoChangeStreamOptions; import org.apache.flink.connector.mongodb.source.config.MongoReadOptions; import org.apache.flink.connector.mongodb.table.config.MongoConfiguration; import org.apache.flink.table.catalog.ResolvedSchema; @@ -36,6 +37,8 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.util.function.SerializableFunction; +import com.mongodb.client.model.changestream.FullDocument; +import com.mongodb.client.model.changestream.FullDocumentBeforeChange; import org.bson.BsonValue; import javax.annotation.Nullable; @@ -45,6 +48,8 @@ import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BUFFER_FLUSH_INTERVAL; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BUFFER_FLUSH_MAX_ROWS; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.CHANGE_STREAM_FETCH_SIZE; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.CHANGE_STREAM_FULL_DOCUMENT_STRATEGY; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.COLLECTION; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE; @@ -54,6 +59,8 @@ import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_PARTITION_SAMPLES; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_PARTITION_SIZE; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_PARTITION_STRATEGY; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_STARTUP_MODE; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_MAX_RETRIES; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.URI; @@ -91,6 +98,10 @@ public Set> optionalOptions() { optionalOptions.add(SCAN_PARTITION_STRATEGY); optionalOptions.add(SCAN_PARTITION_SIZE); optionalOptions.add(SCAN_PARTITION_SAMPLES); + optionalOptions.add(SCAN_STARTUP_MODE); + optionalOptions.add(SCAN_STARTUP_TIMESTAMP_MILLIS); + optionalOptions.add(CHANGE_STREAM_FETCH_SIZE); + optionalOptions.add(CHANGE_STREAM_FULL_DOCUMENT_STRATEGY); optionalOptions.add(BUFFER_FLUSH_MAX_ROWS); optionalOptions.add(BUFFER_FLUSH_INTERVAL); optionalOptions.add(DELIVERY_GUARANTEE); @@ -115,6 +126,10 @@ public Set> forwardOptions() { forwardOptions.add(COLLECTION); forwardOptions.add(SCAN_FETCH_SIZE); forwardOptions.add(SCAN_CURSOR_NO_TIMEOUT); + forwardOptions.add(SCAN_STARTUP_MODE); + forwardOptions.add(SCAN_STARTUP_TIMESTAMP_MILLIS); + forwardOptions.add(CHANGE_STREAM_FETCH_SIZE); + forwardOptions.add(CHANGE_STREAM_FULL_DOCUMENT_STRATEGY); forwardOptions.add(BUFFER_FLUSH_MAX_ROWS); forwardOptions.add(BUFFER_FLUSH_INTERVAL); forwardOptions.add(SINK_MAX_RETRIES); @@ -134,6 +149,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { return new MongoDynamicTableSource( getConnectionOptions(config), getReadOptions(config), + getChangeStreamOptions(config), + config.getScanStartupMode(), getLookupCache(options), config.getLookupMaxRetries(), config.getLookupRetryIntervalMs(), @@ -191,6 +208,29 @@ private static MongoReadOptions getReadOptions(MongoConfiguration configuration) .build(); } + private static MongoChangeStreamOptions getChangeStreamOptions( + MongoConfiguration configuration) { + MongoChangeStreamOptions.MongoChangeStreamOptionsBuilder builder = + MongoChangeStreamOptions.builder() + .setFetchSize(configuration.getChangeStreamFetchSize()); + + switch (configuration.getFullDocumentStrategy()) { + case UPDATE_LOOKUP: + builder.setFullDocument(FullDocument.UPDATE_LOOKUP) + .setFullDocumentBeforeChange(FullDocumentBeforeChange.OFF); + break; + case PRE_AND_POST_IMAGES: + builder.setFullDocument(FullDocument.REQUIRED) + .setFullDocumentBeforeChange(FullDocumentBeforeChange.REQUIRED); + break; + default: + throw new IllegalArgumentException( + "Unsupported fullDocumentStrategy " + + configuration.getFullDocumentStrategy()); + } + return builder.build(); + } + private static MongoWriteOptions getWriteOptions(MongoConfiguration configuration) { return MongoWriteOptions.builder() .setBatchSize(configuration.getBufferFlushMaxRows()) diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java index d3e2a534..57b7256c 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java @@ -21,7 +21,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; import org.apache.flink.connector.mongodb.source.MongoSource; +import org.apache.flink.connector.mongodb.source.config.MongoChangeStreamOptions; import org.apache.flink.connector.mongodb.source.config.MongoReadOptions; +import org.apache.flink.connector.mongodb.source.config.MongoStartupOptions; import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema; import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataDeserializationSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -40,6 +42,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; +import com.mongodb.client.model.changestream.FullDocumentBeforeChange; + import javax.annotation.Nullable; import java.util.ArrayList; @@ -59,7 +63,9 @@ public class MongoDynamicTableSource private final MongoConnectionOptions connectionOptions; private final MongoReadOptions readOptions; + private final MongoStartupOptions startupOptions; @Nullable private final LookupCache lookupCache; + private final MongoChangeStreamOptions changeStreamOptions; private final int lookupMaxRetries; private final long lookupRetryIntervalMs; private DataType producedDataType; @@ -68,12 +74,16 @@ public class MongoDynamicTableSource public MongoDynamicTableSource( MongoConnectionOptions connectionOptions, MongoReadOptions readOptions, + MongoChangeStreamOptions changeStreamOptions, + MongoStartupOptions startupOptions, @Nullable LookupCache lookupCache, int lookupMaxRetries, long lookupRetryIntervalMs, DataType producedDataType) { this.connectionOptions = connectionOptions; this.readOptions = readOptions; + this.startupOptions = startupOptions; + this.changeStreamOptions = changeStreamOptions; this.lookupCache = lookupCache; checkArgument( lookupMaxRetries >= 0, @@ -128,11 +138,16 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon .setUri(connectionOptions.getUri()) .setDatabase(connectionOptions.getDatabase()) .setCollection(connectionOptions.getCollection()) + .setStartupOptions(startupOptions) .setFetchSize(readOptions.getFetchSize()) .setNoCursorTimeout(readOptions.isNoCursorTimeout()) .setPartitionStrategy(readOptions.getPartitionStrategy()) .setPartitionSize(readOptions.getPartitionSize()) .setSamplesPerPartition(readOptions.getSamplesPerPartition()) + .setChangeStreamFetchSize(changeStreamOptions.getFetchSize()) + .setFullDocument(changeStreamOptions.getFullDocument()) + .setFullDocumentBeforeChange( + changeStreamOptions.getFullDocumentBeforeChange()) .setLimit(limit) .setProjectedFields(DataType.getFieldNames(producedDataType)) .setDeserializationSchema(deserializationSchema) @@ -143,7 +158,19 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon @Override public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); + switch (startupOptions.getStartupMode()) { + case INITIAL: + case TIMESTAMP: + case LATEST_OFFSET: + if (changeStreamOptions.getFullDocumentBeforeChange() + == FullDocumentBeforeChange.REQUIRED) { + return ChangelogMode.all(); + } + return ChangelogMode.upsert(); + case BOUNDED: + default: + return ChangelogMode.insertOnly(); + } } @Override @@ -151,6 +178,8 @@ public DynamicTableSource copy() { return new MongoDynamicTableSource( connectionOptions, readOptions, + changeStreamOptions, + startupOptions, lookupCache, lookupMaxRetries, lookupRetryIntervalMs, @@ -186,6 +215,8 @@ public boolean equals(Object o) { MongoDynamicTableSource that = (MongoDynamicTableSource) o; return Objects.equals(connectionOptions, that.connectionOptions) && Objects.equals(readOptions, that.readOptions) + && Objects.equals(changeStreamOptions, that.changeStreamOptions) + && Objects.equals(startupOptions, that.startupOptions) && Objects.equals(producedDataType, that.producedDataType) && Objects.equals(limit, that.limit) && Objects.equals(lookupCache, that.lookupCache) @@ -198,6 +229,8 @@ public int hashCode() { return Objects.hash( connectionOptions, readOptions, + changeStreamOptions, + startupOptions, producedDataType, limit, lookupCache, diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java index b3336488..6bc404ec 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java @@ -30,7 +30,6 @@ import com.mongodb.MongoException; import com.mongodb.client.MongoClient; -import com.mongodb.client.MongoClients; import com.mongodb.client.MongoCollection; import com.mongodb.client.MongoCursor; import org.bson.BsonDocument; @@ -47,6 +46,7 @@ import static com.mongodb.client.model.Filters.and; import static com.mongodb.client.model.Filters.eq; +import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.clientFor; import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.project; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -99,7 +99,7 @@ public MongoRowDataLookupFunction( @Override public void open(FunctionContext context) { - this.mongoClient = MongoClients.create(connectionOptions.getUri()); + this.mongoClient = clientFor(connectionOptions); } /** diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/FullDocumentStrategy.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/FullDocumentStrategy.java new file mode 100644 index 00000000..38f9f4d4 --- /dev/null +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/FullDocumentStrategy.java @@ -0,0 +1,66 @@ +/* + * 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.flink.connector.mongodb.table.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.DescribedEnum; +import org.apache.flink.configuration.description.InlineElement; + +import static org.apache.flink.configuration.description.TextElement.text; + +/** + * FullDocument strategies that can be chosen. Available strategies are update-lookup, and + * pre-and-post-images. + * + *

    + *
  • update-lookup: prior to version 6.0 of MongoDB, the pre images was not stored in the oplog. + * To obtain complete row data, we can use updateLookup feature to lookup the latest snapshot + * when the oplog record is accessed. + *
  • pre-and-post-images: starting in MongoDB 6.0, you can use change stream events to output + * the version of a document before and after changes (the document pre- and post-images). + *
+ */ +@PublicEvolving +public enum FullDocumentStrategy implements DescribedEnum { + UPDATE_LOOKUP( + "update-lookup", + text( + "Prior to version 6.0 of MongoDB, the pre images was not stored in the oplog. To obtain complete row data, we can use updateLookup feature to lookup the latest snapshot when the oplog record is accessed.")), + PRE_AND_POST_IMAGES( + "pre-and-post-images", + text( + "Starting in MongoDB 6.0, you can use change stream events to output the version of a document before and after changes (the document pre- and post-images).")); + + private final String name; + private final InlineElement description; + + FullDocumentStrategy(String name, InlineElement description) { + this.name = name; + this.description = description; + } + + @Override + public InlineElement getDescription() { + return description; + } + + @Override + public String toString() { + return name; + } +} diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/MongoConfiguration.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/MongoConfiguration.java index c141ca25..cc45c570 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/MongoConfiguration.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/MongoConfiguration.java @@ -21,7 +21,9 @@ import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.mongodb.source.config.MongoStartupOptions; import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.source.lookup.LookupOptions; import javax.annotation.Nullable; @@ -30,6 +32,8 @@ import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BUFFER_FLUSH_INTERVAL; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BUFFER_FLUSH_MAX_ROWS; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.CHANGE_STREAM_FETCH_SIZE; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.CHANGE_STREAM_FULL_DOCUMENT_STRATEGY; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.COLLECTION; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE; @@ -39,10 +43,13 @@ import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_PARTITION_SAMPLES; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_PARTITION_SIZE; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_PARTITION_STRATEGY; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_STARTUP_MODE; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_MAX_RETRIES; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.URI; import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.apache.flink.util.Preconditions.checkNotNull; /** MongoDB configuration. */ @Internal @@ -88,6 +95,36 @@ public int getSamplesPerPartition() { return config.get(SCAN_PARTITION_SAMPLES); } + // -----------------------------------Change Stream Config---------------------------------- + public int getChangeStreamFetchSize() { + return config.get(CHANGE_STREAM_FETCH_SIZE); + } + + public FullDocumentStrategy getFullDocumentStrategy() { + return config.get(CHANGE_STREAM_FULL_DOCUMENT_STRATEGY); + } + + // -----------------------------------Startup Config---------------------------------------- + public MongoStartupOptions getScanStartupMode() { + switch (config.get(SCAN_STARTUP_MODE)) { + case BOUNDED: + return MongoStartupOptions.bounded(); + case INITIAL: + return MongoStartupOptions.initial(); + case LATEST_OFFSET: + return MongoStartupOptions.latest(); + case TIMESTAMP: + Long startupTimestampMillis = config.get(SCAN_STARTUP_TIMESTAMP_MILLIS); + checkNotNull( + startupTimestampMillis, + "The startupTimestampMillis shouldn't be null when using timestamp startup mode."); + return MongoStartupOptions.timestamp(startupTimestampMillis); + default: + throw new ValidationException( + "Unknown startup mode of " + config.get(SCAN_STARTUP_MODE)); + } + } + // -----------------------------------Lookup Config---------------------------------------- public int getLookupMaxRetries() { return config.get(LookupOptions.MAX_RETRIES); diff --git a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/serialization/MongoRowDataDeserializationSchema.java b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/serialization/MongoRowDataDeserializationSchema.java index 48bdec8b..3e5ef0ab 100644 --- a/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/serialization/MongoRowDataDeserializationSchema.java +++ b/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/serialization/MongoRowDataDeserializationSchema.java @@ -19,13 +19,25 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema; import org.apache.flink.connector.mongodb.table.converter.BsonToRowDataConverters; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Collector; +import com.mongodb.client.model.changestream.OperationType; import org.bson.BsonDocument; +import java.util.NoSuchElementException; +import java.util.Optional; + +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.DOCUMENT_KEY_FIELD; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.FULL_DOCUMENT_BEFORE_CHANGE_FIELD; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.FULL_DOCUMENT_FIELD; +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OPERATION_TYPE_FIELD; + /** Deserializer that maps {@link BsonDocument} to {@link RowData}. */ @Internal public class MongoRowDataDeserializationSchema implements MongoDeserializationSchema { @@ -42,12 +54,97 @@ public MongoRowDataDeserializationSchema(RowType rowType, TypeInformation out) { + switch (sourceRecord.getType()) { + case SNAPSHOT: + out.collect(runtimeConverter.convert(sourceRecord.getRecord())); + break; + case STREAM: + deserializeStreamRecord(sourceRecord, out); + break; + default: + throw new IllegalStateException( + "Unsupported record type " + sourceRecord.getType()); + } + } + + private void deserializeStreamRecord(MongoSourceRecord sourceRecord, Collector out) { + BsonDocument changeStreamDocument = sourceRecord.getRecord(); + OperationType operationType = getOperationType(changeStreamDocument); + switch (operationType) { + case INSERT: + { + BsonDocument fullDocument = + getFullDocument(changeStreamDocument) + .orElseThrow(NoSuchElementException::new); + RowData insert = runtimeConverter.convert(fullDocument); + insert.setRowKind(RowKind.INSERT); + out.collect(insert); + break; + } + case UPDATE: + case REPLACE: + { + Optional updateBefore = + getFullDocumentBeforeChange(changeStreamDocument); + updateBefore.ifPresent( + doc -> { + RowData before = runtimeConverter.convert(doc); + before.setRowKind(RowKind.UPDATE_BEFORE); + out.collect(before); + }); + + Optional fullDocument = getFullDocument(changeStreamDocument); + fullDocument.ifPresent( + doc -> { + RowData after = runtimeConverter.convert(doc); + after.setRowKind(RowKind.UPDATE_AFTER); + out.collect(after); + }); + break; + } + case DELETE: + { + BsonDocument document = + getFullDocumentBeforeChange(changeStreamDocument) + .orElse(getDocumentKey(changeStreamDocument)); + RowData delete = runtimeConverter.convert(document); + delete.setRowKind(RowKind.DELETE); + out.collect(delete); + break; + } + default: + // ignore ddl and other record + break; + } } @Override public TypeInformation getProducedType() { return typeInfo; } + + private static OperationType getOperationType(BsonDocument changeStreamDocument) { + return OperationType.fromString( + changeStreamDocument.getString(OPERATION_TYPE_FIELD).getValue()); + } + + private static BsonDocument getDocumentKey(BsonDocument changeStreamDocument) { + return changeStreamDocument.getDocument(DOCUMENT_KEY_FIELD); + } + + private static Optional getFullDocument(BsonDocument changeStreamDocument) { + if (changeStreamDocument.containsKey(FULL_DOCUMENT_FIELD)) { + return Optional.of(changeStreamDocument.getDocument(FULL_DOCUMENT_FIELD)); + } + return Optional.empty(); + } + + private static Optional getFullDocumentBeforeChange( + BsonDocument changeStreamDocument) { + if (changeStreamDocument.containsKey(FULL_DOCUMENT_BEFORE_CHANGE_FIELD)) { + return Optional.of(changeStreamDocument.getDocument(FULL_DOCUMENT_BEFORE_CHANGE_FIELD)); + } + return Optional.empty(); + } } diff --git a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoSourceITCase.java b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoSourceITCase.java index e5770060..57c3f0a8 100644 --- a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoSourceITCase.java +++ b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoSourceITCase.java @@ -24,6 +24,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.MemorySize; import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy; +import org.apache.flink.connector.mongodb.source.reader.MongoSourceRecord; import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema; import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataDeserializationSchema; import org.apache.flink.connector.mongodb.testutils.MongoShardedContainers; @@ -40,6 +41,7 @@ import org.apache.flink.testutils.junit.SharedObjectsExtension; import org.apache.flink.testutils.junit.SharedReference; import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.Collector; import com.mongodb.client.MongoClient; import com.mongodb.client.MongoClients; @@ -66,7 +68,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Stream; @@ -74,7 +75,7 @@ import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD; import static org.assertj.core.api.Assertions.assertThat; -/** IT cases for using Mongo Sink. */ +/** IT cases for using Mongo Source. */ @Testcontainers public class MongoSourceITCase { @@ -324,10 +325,8 @@ private static class MongoJsonDeserializationSchema implements MongoDeserializationSchema { @Override - public String deserialize(BsonDocument document) { - return Optional.ofNullable(document) - .map(doc -> doc.toJson(DEFAULT_JSON_WRITER_SETTINGS)) - .orElse(null); + public void deserialize(MongoSourceRecord sourceRecord, Collector out) { + out.collect(sourceRecord.getRecord().toJson(DEFAULT_JSON_WRITER_SETTINGS)); } @Override diff --git a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoUnboundedSourceITCase.java b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoUnboundedSourceITCase.java new file mode 100644 index 00000000..233669c8 --- /dev/null +++ b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoUnboundedSourceITCase.java @@ -0,0 +1,393 @@ +/* + * 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.flink.connector.mongodb.source; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.connector.mongodb.source.config.MongoStartupOptions; +import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataDeserializationSchema; +import org.apache.flink.connector.mongodb.testutils.MongoTestUtil; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; +import org.apache.flink.types.RowUtils; +import org.apache.flink.util.CloseableIterator; + +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.ChangeStreamPreAndPostImagesOptions; +import com.mongodb.client.model.CreateCollectionOptions; +import com.mongodb.client.model.Filters; +import com.mongodb.client.model.Updates; +import com.mongodb.client.model.changestream.FullDocument; +import com.mongodb.client.model.changestream.FullDocumentBeforeChange; +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonString; +import org.bson.conversions.Bson; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.MongoDBContainer; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_4_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_5_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_6_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_IMAGE_PREFIX; +import static org.assertj.core.api.Assertions.assertThat; + +/** IT cases for using Mongo Source unbounded stream reading. */ +@ExtendWith(ParameterizedTestExtension.class) +public class MongoUnboundedSourceITCase { + + private static final Logger LOG = LoggerFactory.getLogger(MongoUnboundedSourceITCase.class); + + private static final int PARALLELISM = 2; + + private static final String TEST_DATABASE = "test_stream_source"; + private static final String TEST_COLLECTION = "test_stream_coll"; + + private static final int TEST_RECORD_SIZE = 30000; + private static final int TEST_RECORD_BATCH_SIZE = 10000; + + @Parameter public String mongoVersion; + + @Parameter(value = 1) + public FullDocument fullDocument; + + @Parameter(value = 2) + public FullDocumentBeforeChange fullDocumentBeforeChange; + + @Parameters(name = "mongoVersion={0} fullDocument={1} fullDocumentBeforeChange={2}") + public static List parameters() { + return Arrays.asList( + new Object[] {MONGO_4_0, FullDocument.UPDATE_LOOKUP, FullDocumentBeforeChange.OFF}, + new Object[] {MONGO_5_0, FullDocument.UPDATE_LOOKUP, FullDocumentBeforeChange.OFF}, + new Object[] {MONGO_6_0, FullDocument.UPDATE_LOOKUP, FullDocumentBeforeChange.OFF}, + new Object[] {MONGO_6_0, FullDocument.REQUIRED, FullDocumentBeforeChange.REQUIRED}); + } + + @RegisterExtension + static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .withHaLeadershipControl() + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .build()); + + private MongoDBContainer mongoDBContainer; + private MongoClient mongoClient; + private MongoCollection testCollection; + + @BeforeEach + void before() { + mongoDBContainer = + MongoTestUtil.createMongoDBContainer(MONGO_IMAGE_PREFIX + mongoVersion, LOG); + mongoDBContainer.start(); + mongoClient = MongoClients.create(mongoDBContainer.getConnectionString()); + + MongoDatabase testDatabase = mongoClient.getDatabase(TEST_DATABASE); + if (MONGO_6_0.equals(mongoVersion)) { + testDatabase.createCollection( + TEST_COLLECTION, + new CreateCollectionOptions() + .changeStreamPreAndPostImagesOptions( + new ChangeStreamPreAndPostImagesOptions(true))); + } + testCollection = + testDatabase.getCollection(TEST_COLLECTION).withDocumentClass(BsonDocument.class); + initTestData(); + } + + @AfterEach + void after() { + mongoClient.close(); + mongoDBContainer.close(); + } + + @TestTemplate + void testInitialStartup(@InjectMiniCluster MiniCluster miniCluster) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(300L); + + MongoSource mongoSource = createSource(MongoStartupOptions.initial()); + + DataStreamSource source = + env.fromSource(mongoSource, WatermarkStrategy.noWatermarks(), "MongoDB-Source"); + + CloseableIterator iterator = source.collectAsync(); + JobClient jobClient = env.executeAsync(); + + // Trigger failover once some snapshot records has been sent by sleeping source + if (iterator.hasNext()) { + triggerFailover( + FailoverType.JM, jobClient.getJobID(), () -> sleepMs(1000), miniCluster); + } + + List initialRows = fetchRowData(iterator, TEST_RECORD_SIZE); + // assert scanned rows + assertThat(initialRows).hasSize(TEST_RECORD_SIZE); + + // assert changed rows + generateAndAssertStreamChanges(miniCluster, jobClient, iterator); + } + + @TestTemplate + void testTimestampStartup(@InjectMiniCluster MiniCluster miniCluster) throws Exception { + // Unfortunately we need this sleep here to make sure we could differ the coming oplog + // events from existing events by timestamp. + sleepMs(2000L); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(300L); + + MongoSource mongoSource = + createSource(MongoStartupOptions.timestamp(System.currentTimeMillis())); + + DataStreamSource source = + env.fromSource(mongoSource, WatermarkStrategy.noWatermarks(), "MongoDB-Source"); + + CloseableIterator iterator = source.collectAsync(); + JobClient jobClient = env.executeAsync(); + + generateAndAssertStreamChanges(miniCluster, jobClient, iterator); + } + + @TestTemplate + void testLatestStartup(@InjectMiniCluster MiniCluster miniCluster) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(300L); + + MongoSource mongoSource = createSource(MongoStartupOptions.latest()); + DataStreamSource source = + env.fromSource(mongoSource, WatermarkStrategy.noWatermarks(), "MongoDB-Source"); + + CloseableIterator iterator = source.collectAsync(); + JobClient jobClient = env.executeAsync(); + + // Unfortunately we need this sleep here to make sure we could differ the coming oplog + // events from existing events by timestamp. + sleepMs(2000L); + + generateAndAssertStreamChanges(miniCluster, jobClient, iterator); + } + + private void generateAndAssertStreamChanges( + MiniCluster miniCluster, JobClient jobClient, CloseableIterator iterator) + throws Exception { + // insert 3 + int index = TEST_RECORD_SIZE + 1; + List inserts = + Arrays.asList( + createTestData(index), + createTestData(index + 1), + createTestData(index + 2)); + testCollection.insertMany(inserts); + + // delete 1 + testCollection.deleteOne(Filters.eq(ID_FIELD, inserts.get(2).getInt32(ID_FIELD))); + + triggerFailover(FailoverType.TM, jobClient.getJobID(), () -> sleepMs(300L), miniCluster); + + // update 1 + testCollection.updateOne( + Filters.eq(ID_FIELD, inserts.get(1).getInt32(ID_FIELD)), + Updates.set("name", new BsonString("name_updated"))); + + // replace 1 + Bson filter = Filters.eq(ID_FIELD, inserts.get(0).getInt32(ID_FIELD)); + BsonDocument replacement = + new BsonDocument("name", new BsonString("name_replaced")) + .append("address", new BsonString("address_replaced")); + testCollection.replaceOne(filter, replacement); + + if (fullDocumentBeforeChange == FullDocumentBeforeChange.REQUIRED) { + // all changelog mode + List allRows = fetchRowData(iterator, 8); + assertThat(allRows) + .containsExactly( + "+I[30001, name_30001, address_30001]", + "+I[30002, name_30002, address_30002]", + "+I[30003, name_30003, address_30003]", + "-D[30003, name_30003, address_30003]", + "-U[30002, name_30002, address_30002]", + "+U[30002, name_updated, address_30002]", + "-U[30001, name_30001, address_30001]", + "+U[30001, name_replaced, address_replaced]"); + } else { + // upsert changelog mode + List upsertRows = fetchRowData(iterator, 6); + assertThat(upsertRows) + .containsExactly( + "+I[30001, name_30001, address_30001]", + "+I[30002, name_30002, address_30002]", + "+I[30003, name_30003, address_30003]", + "-D[30003, , ]", + "+U[30002, name_updated, address_30002]", + "+U[30001, name_replaced, address_replaced]"); + } + + iterator.close(); + } + + private MongoSource createSource(MongoStartupOptions startupOptions) { + ResolvedSchema schema = + ResolvedSchema.of( + Column.physical("_id", DataTypes.INT()), + Column.physical("name", DataTypes.STRING()), + Column.physical("address", DataTypes.STRING())); + + RowType rowType = (RowType) schema.toPhysicalRowDataType().getLogicalType(); + TypeInformation typeInfo = InternalTypeInfo.of(rowType); + + return MongoSource.builder() + .setUri(mongoDBContainer.getConnectionString()) + .setDatabase(TEST_DATABASE) + .setCollection(TEST_COLLECTION) + .setStartupOptions(startupOptions) + .setPartitionSize(MemorySize.parse("1mb")) + .setFullDocument(fullDocument) + .setFullDocumentBeforeChange(fullDocumentBeforeChange) + .setDeserializationSchema(new MongoRowDataDeserializationSchema(rowType, typeInfo)) + .build(); + } + + private void initTestData() { + List testRecords = new ArrayList<>(); + for (int i = 1; i <= TEST_RECORD_SIZE; i++) { + testRecords.add(createTestData(i)); + if (testRecords.size() >= TEST_RECORD_BATCH_SIZE) { + testCollection.insertMany(testRecords); + testRecords.clear(); + } + } + } + + private static BsonDocument createTestData(int id) { + return new BsonDocument("_id", new BsonInt32(id)) + .append("name", new BsonString("name_" + id)) + .append("address", new BsonString("address_" + id)); + } + + private static List fetchRowData(Iterator iter, int size) { + List rows = new ArrayList<>(size); + while (size > 0 && iter.hasNext()) { + RowData row = iter.next(); + rows.add(row); + size--; + } + return convertRowDataToRowString(rows); + } + + private static List convertRowDataToRowString(List rows) { + LinkedHashMap map = new LinkedHashMap<>(); + map.put("_id", 0); + map.put("name", 1); + map.put("address", 2); + return rows.stream() + .map( + row -> + RowUtils.createRowWithNamedPositions( + row.getRowKind(), + new Object[] { + row.getInt(0), + row.getString(1), + row.getString(2) + }, + map) + .toString()) + .collect(Collectors.toList()); + } + + private enum FailoverType { + NONE, + TM, + JM + } + + private static void triggerFailover( + FailoverType type, JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) + throws Exception { + switch (type) { + case NONE: + afterFailAction.run(); + break; + case TM: + restartTaskManager(afterFailAction, miniCluster); + break; + case JM: + triggerJobManagerFailover(jobId, afterFailAction, miniCluster); + break; + } + } + + private static void triggerJobManagerFailover( + JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) throws Exception { + final HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); + haLeadershipControl.revokeJobMasterLeadership(jobId).get(); + afterFailAction.run(); + haLeadershipControl.grantJobMasterLeadership(jobId).get(); + } + + private static void restartTaskManager(Runnable afterFailAction, MiniCluster miniCluster) + throws Exception { + miniCluster.terminateTaskManager(0).get(); + afterFailAction.run(); + miniCluster.startTaskManager(); + } + + private static void sleepMs(long millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException ignored) { + } + } +} diff --git a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializerTest.java b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializerTest.java index 001c10e3..d8b02b40 100644 --- a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializerTest.java +++ b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializerTest.java @@ -40,6 +40,7 @@ public class MongoSourceEnumStateSerializerTest { @Test void serializeAndDeserializeMongoSourceEnumState() throws Exception { boolean initialized = false; + boolean isStreamSplitAssigned = false; List remainingCollections = Arrays.asList("db.remains0", "db.remains1"); List alreadyProcessedCollections = Arrays.asList("db.processed0", "db.processed1"); List remainingScanSplits = @@ -54,7 +55,8 @@ void serializeAndDeserializeMongoSourceEnumState() throws Exception { alreadyProcessedCollections, remainingScanSplits, assignedScanSplits, - initialized); + initialized, + isStreamSplitAssigned); byte[] bytes = INSTANCE.serialize(state); MongoSourceEnumState state1 = INSTANCE.deserialize(INSTANCE.getVersion(), bytes); @@ -65,6 +67,7 @@ void serializeAndDeserializeMongoSourceEnumState() throws Exception { assertEquals(state.getRemainingScanSplits(), state1.getRemainingScanSplits()); assertEquals(state.getAssignedScanSplits(), state1.getAssignedScanSplits()); assertEquals(state.isInitialized(), state1.isInitialized()); + assertEquals(state.isStreamSplitAssigned(), state1.isStreamSplitAssigned()); assertNotSame(state, state1); } diff --git a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitSerializerTest.java b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitSerializerTest.java new file mode 100644 index 00000000..e4f00a2e --- /dev/null +++ b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitSerializerTest.java @@ -0,0 +1,67 @@ +/* + * 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.flink.connector.mongodb.source.split; + +import org.apache.flink.connector.mongodb.common.utils.MongoConstants; + +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT; +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit tests for {@link MongoSourceSplitSerializer}. */ +class MongoSourceSplitSerializerTest { + + @Test + void serializeAndDeserializeMongoScanSourceSplit() throws IOException { + MongoScanSourceSplit scanSourceSplit = createScanSourceSplit(); + byte[] bytes = MongoSourceSplitSerializer.INSTANCE.serialize(scanSourceSplit); + MongoSourceSplit deserialized = MongoSourceSplitSerializer.INSTANCE.deserialize(1, bytes); + assertThat(deserialized).isEqualTo(scanSourceSplit); + } + + @Test + void serializeAndDeserializeMongoStreamSourceSplit() throws IOException { + MongoStreamSourceSplit streamSourceSplit = createStreamSourceSplit(); + byte[] bytes = MongoSourceSplitSerializer.INSTANCE.serialize(streamSourceSplit); + MongoSourceSplit deserialized = MongoSourceSplitSerializer.INSTANCE.deserialize(1, bytes); + assertThat(deserialized).isEqualTo(streamSourceSplit); + } + + private static MongoScanSourceSplit createScanSourceSplit() { + return new MongoScanSourceSplit( + "split", + "db", + "coll", + new BsonDocument("_id", new BsonInt32(0)), + new BsonDocument("_id", MongoConstants.BSON_MAX_KEY), + ID_HINT); + } + + private static MongoStreamSourceSplit createStreamSourceSplit() { + return new MongoStreamSourceSplit( + "split", + "db", + "coll", + MongoStreamOffset.fromTimeMillis(System.currentTimeMillis())); + } +} diff --git a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java index e2337f3c..2d247943 100644 --- a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java +++ b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java @@ -22,8 +22,11 @@ import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions; import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions; +import org.apache.flink.connector.mongodb.source.config.MongoChangeStreamOptions; import org.apache.flink.connector.mongodb.source.config.MongoReadOptions; +import org.apache.flink.connector.mongodb.source.config.MongoStartupOptions; import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy; +import org.apache.flink.connector.mongodb.table.config.FullDocumentStrategy; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ResolvedSchema; @@ -33,7 +36,11 @@ import org.apache.flink.table.connector.source.lookup.LookupOptions; import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache; +import com.mongodb.client.model.changestream.FullDocument; +import com.mongodb.client.model.changestream.FullDocumentBeforeChange; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import java.util.Arrays; import java.util.Collections; @@ -43,6 +50,8 @@ import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BUFFER_FLUSH_INTERVAL; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BUFFER_FLUSH_MAX_ROWS; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.CHANGE_STREAM_FETCH_SIZE; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.CHANGE_STREAM_FULL_DOCUMENT_STRATEGY; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.COLLECTION; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE; @@ -52,6 +61,8 @@ import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_PARTITION_SAMPLES; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_PARTITION_SIZE; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_PARTITION_STRATEGY; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_STARTUP_MODE; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_STARTUP_TIMESTAMP_MILLIS; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_MAX_RETRIES; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.URI; @@ -86,6 +97,8 @@ public void testMongoSourceCommonProperties() { new MongoDynamicTableSource( getConnectionOptions(), MongoReadOptions.builder().build(), + MongoChangeStreamOptions.builder().build(), + MongoStartupOptions.bounded(), null, LookupOptions.MAX_RETRIES.defaultValue(), LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(), @@ -116,6 +129,8 @@ public void testMongoReadProperties() { properties.put(SCAN_PARTITION_STRATEGY.key(), "split-vector"); properties.put(SCAN_PARTITION_SIZE.key(), "128m"); properties.put(SCAN_PARTITION_SAMPLES.key(), "5"); + properties.put(SCAN_STARTUP_MODE.key(), "timestamp"); + properties.put(SCAN_STARTUP_TIMESTAMP_MILLIS.key(), "1686902599000"); DynamicTableSource actual = createTableSource(SCHEMA, properties); @@ -133,6 +148,8 @@ public void testMongoReadProperties() { new MongoDynamicTableSource( connectionOptions, readOptions, + MongoChangeStreamOptions.builder().build(), + MongoStartupOptions.timestamp(1686902599000L), null, LookupOptions.MAX_RETRIES.defaultValue(), LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(), @@ -160,6 +177,8 @@ public void testMongoLookupProperties() { new MongoDynamicTableSource( connectionOptions, MongoReadOptions.builder().build(), + MongoChangeStreamOptions.builder().build(), + MongoStartupOptions.bounded(), DefaultLookupCache.fromConfig(Configuration.fromMap(properties)), 10, 20, @@ -168,6 +187,54 @@ public void testMongoLookupProperties() { assertThat(actual).isEqualTo(expected); } + @ParameterizedTest + @EnumSource(value = FullDocumentStrategy.class) + public void testMongoChangeStreamProperties(FullDocumentStrategy fullDocumentStrategy) { + Map properties = getRequiredOptions(); + properties.put(CHANGE_STREAM_FETCH_SIZE.key(), "10"); + properties.put(CHANGE_STREAM_FULL_DOCUMENT_STRATEGY.key(), fullDocumentStrategy.toString()); + + DynamicTableSource actual = createTableSource(SCHEMA, properties); + + MongoConnectionOptions connectionOptions = getConnectionOptions(); + + MongoChangeStreamOptions changeStreamOptions; + switch (fullDocumentStrategy) { + case PRE_AND_POST_IMAGES: + changeStreamOptions = + MongoChangeStreamOptions.builder() + .setFetchSize(10) + .setFullDocument(FullDocument.REQUIRED) + .setFullDocumentBeforeChange(FullDocumentBeforeChange.REQUIRED) + .build(); + break; + case UPDATE_LOOKUP: + changeStreamOptions = + MongoChangeStreamOptions.builder() + .setFetchSize(10) + .setFullDocument(FullDocument.UPDATE_LOOKUP) + .setFullDocumentBeforeChange(FullDocumentBeforeChange.OFF) + .build(); + break; + default: + throw new IllegalStateException( + "Unknown fullDocumentStrategy " + fullDocumentStrategy); + } + + MongoDynamicTableSource expected = + new MongoDynamicTableSource( + connectionOptions, + MongoReadOptions.builder().build(), + changeStreamOptions, + MongoStartupOptions.bounded(), + null, + LookupOptions.MAX_RETRIES.defaultValue(), + LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(), + SCHEMA.toPhysicalRowDataType()); + + assertThat(actual).isEqualTo(expected); + } + @Test public void testMongoSinkProperties() { Map properties = getRequiredOptions(); @@ -242,6 +309,18 @@ public void testMongoValidation() { "0", "The samples per partition must be larger than 0."); + // change stream fetch size lower than 1 + assertSourceValidationRejects( + CHANGE_STREAM_FETCH_SIZE.key(), + "0", + "The change stream fetch size must be larger than 0."); + + // startup timestamp millis shouldn't be null when using timestamp startup mode. + assertSourceValidationRejects( + SCAN_STARTUP_MODE.key(), + "timestamp", + "The startupTimestampMillis shouldn't be null when using timestamp startup mode."); + // lookup retry times shouldn't be negative assertSourceValidationRejects( LookupOptions.MAX_RETRIES.key(), diff --git a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSourceITCase.java b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSourceITCase.java index e55ef6f1..83852f9e 100644 --- a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSourceITCase.java +++ b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSourceITCase.java @@ -17,24 +17,16 @@ package org.apache.flink.connector.mongodb.table; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.connector.mongodb.table.config.FullDocumentStrategy; import org.apache.flink.connector.mongodb.testutils.MongoTestUtil; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.connector.source.lookup.LookupOptions; -import org.apache.flink.table.connector.source.lookup.cache.LookupCache; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.functions.table.lookup.LookupCacheManager; -import org.apache.flink.table.test.lookup.cache.LookupCacheAssert; import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.CollectionUtil; @@ -42,6 +34,11 @@ import com.mongodb.client.MongoClient; import com.mongodb.client.MongoClients; import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.ChangeStreamPreAndPostImagesOptions; +import com.mongodb.client.model.CreateCollectionOptions; +import com.mongodb.client.model.Filters; +import com.mongodb.client.model.Updates; import org.bson.BsonArray; import org.bson.BsonBinary; import org.bson.BsonBoolean; @@ -54,22 +51,18 @@ import org.bson.BsonString; import org.bson.BsonTimestamp; import org.bson.types.Decimal128; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.MongoDBContainer; -import org.testcontainers.junit.jupiter.Container; -import org.testcontainers.junit.jupiter.Testcontainers; import java.math.BigDecimal; +import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -80,14 +73,20 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.CHANGE_STREAM_FULL_DOCUMENT_STRATEGY; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.COLLECTION; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_STARTUP_MODE; import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.URI; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_4_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_5_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_6_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_IMAGE_PREFIX; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; import static org.assertj.core.api.Assertions.assertThat; /** ITCase for {@link MongoDynamicTableSource}. */ -@Testcontainers +@ExtendWith(ParameterizedTestExtension.class) public class MongoDynamicTableSourceITCase { private static final Logger LOG = LoggerFactory.getLogger(MongoDynamicTableSinkITCase.class); @@ -99,46 +98,58 @@ public class MongoDynamicTableSourceITCase { .setNumberTaskManagers(1) .build()); - @Container - private static final MongoDBContainer MONGO_CONTAINER = - MongoTestUtil.createMongoDBContainer(LOG); + @Parameter public String mongoVersion; + + @Parameters(name = "mongoVersion={0}") + public static List parameters() { + return Arrays.asList(MONGO_4_0, MONGO_5_0, MONGO_6_0); + } private static final String TEST_DATABASE = "test"; private static final String TEST_COLLECTION = "mongo_table_source"; - private static MongoClient mongoClient; + private MongoDBContainer mongoDBContainer; + private MongoClient mongoClient; + private MongoDatabase testDatabase; + private MongoCollection testCollection; - public static StreamExecutionEnvironment env; - public static StreamTableEnvironment tEnv; + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; - @BeforeAll - static void beforeAll() { - mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString()); - - MongoCollection coll = - mongoClient - .getDatabase(TEST_DATABASE) - .getCollection(TEST_COLLECTION) - .withDocumentClass(BsonDocument.class); + @BeforeEach + void before() { + mongoDBContainer = + MongoTestUtil.createMongoDBContainer(MONGO_IMAGE_PREFIX + mongoVersion, LOG); + mongoDBContainer.start(); + mongoClient = MongoClients.create(mongoDBContainer.getConnectionString()); + testDatabase = mongoClient.getDatabase(TEST_DATABASE); + testCollection = + testDatabase.getCollection(TEST_COLLECTION).withDocumentClass(BsonDocument.class); + + if (MONGO_6_0.equals(mongoVersion)) { + testDatabase.createCollection( + TEST_COLLECTION, + new CreateCollectionOptions() + .changeStreamPreAndPostImagesOptions( + new ChangeStreamPreAndPostImagesOptions(true))); + } List testRecords = Arrays.asList(createTestData(1), createTestData(2)); - coll.insertMany(testRecords); + testCollection.insertMany(testRecords); + + env = StreamExecutionEnvironment.getExecutionEnvironment(); + tEnv = StreamTableEnvironment.create(env); } - @AfterAll - static void afterAll() { + @AfterEach + void after() { if (mongoClient != null) { mongoClient.close(); } + mongoDBContainer.close(); } - @BeforeEach - void before() { - env = StreamExecutionEnvironment.getExecutionEnvironment(); - tEnv = StreamTableEnvironment.create(env); - } - - @Test + @TestTemplate public void testSource() { tEnv.executeSql(createTestDDl(null)); @@ -159,7 +170,80 @@ public void testSource() { assertThat(result).isEqualTo(expected); } - @Test + @TestTemplate + public void testUnboundedSource() throws Exception { + // Create MongoDB lookup table + Map options = new HashMap<>(); + options.put( + SCAN_STARTUP_MODE.key(), MongoConnectorOptions.ScanStartupMode.INITIAL.toString()); + if (MONGO_6_0.equals(mongoVersion)) { + options.put( + CHANGE_STREAM_FULL_DOCUMENT_STRATEGY.key(), + FullDocumentStrategy.PRE_AND_POST_IMAGES.toString()); + } + + tEnv.executeSql(createTestDDl(options)); + TableResult tableResult = tEnv.executeSql("SELECT * FROM mongo_source"); + + try (CloseableIterator iterator = tableResult.collect()) { + // fetch scan records + List scanned = fetchRows(iterator, 2); + List expected = + Arrays.asList( + "+I[1, 2, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]", + "+I[2, 2, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]"); + // assert scanned records + assertThat(scanned).containsExactlyElementsOf(expected); + + // insert 2 records + List newDocs = + Arrays.asList(createTestData(3), createTestData(4), createTestData(5)); + testCollection.insertMany(newDocs); + List inserted = fetchRows(iterator, 3); + expected = + Arrays.asList( + "+I[3, 2, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]", + "+I[4, 2, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]", + "+I[5, 2, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]"); + // assert inserted records + assertThat(inserted).containsExactlyElementsOf(expected); + + // delete 1 record + testCollection.deleteOne(Filters.eq("_id", newDocs.get(0).getInt64("_id"))); + List deleted = fetchRows(iterator, 1); + expected = + Collections.singletonList( + "-D[3, 2, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]"); + // assert deleted records + assertThat(deleted).containsExactlyElementsOf(expected); + + // update 1 record + testCollection.updateOne( + Filters.eq("_id", newDocs.get(1).getInt64("_id")), + Updates.set("f1", new BsonString("3"))); + List updated = fetchRows(iterator, 2); + expected = + Arrays.asList( + "-U[4, 2, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]", + "+U[4, 3, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]"); + // assert updated records + assertThat(updated).containsExactlyElementsOf(expected); + + // replace 1 record + BsonDocument replacement = newDocs.get(2); + replacement.put("f1", new BsonString("4")); + testCollection.replaceOne(Filters.eq("_id", replacement.remove("_id")), replacement); + List replaced = fetchRows(iterator, 2); + expected = + Arrays.asList( + "-U[5, 2, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]", + "+U[5, 4, false, [3], 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.10, {k=12}, +I[13], [11_1, 11_2], [+I[12_1], +I[12_2]]]"); + // assert replaced records + assertThat(replaced).containsExactlyElementsOf(expected); + } + } + + @TestTemplate public void testProject() { tEnv.executeSql(createTestDDl(null)); @@ -176,7 +260,7 @@ public void testProject() { assertThat(result).isEqualTo(expected); } - @Test + @TestTemplate public void testLimit() { tEnv.executeSql(createTestDDl(null)); @@ -197,111 +281,10 @@ public void testLimit() { assertThat(result).containsAnyElementsOf(expected); } - @ParameterizedTest - @EnumSource(Caching.class) - public void testLookupJoin(Caching caching) throws Exception { - // Create MongoDB lookup table - Map lookupOptions = new HashMap<>(); - if (caching.equals(Caching.ENABLE_CACHE)) { - lookupOptions.put(LookupOptions.CACHE_TYPE.key(), "PARTIAL"); - lookupOptions.put(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE.key(), "10min"); - lookupOptions.put(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS.key(), "10min"); - lookupOptions.put(LookupOptions.PARTIAL_CACHE_MAX_ROWS.key(), "100"); - lookupOptions.put(LookupOptions.MAX_RETRIES.key(), "10"); - } - - tEnv.executeSql(createTestDDl(lookupOptions)); - - DataStream sourceStream = - env.fromCollection( - Arrays.asList( - Row.of(1L, "Alice"), - Row.of(1L, "Alice"), - Row.of(2L, "Bob"), - Row.of(3L, "Charlie"))) - .returns( - new RowTypeInfo( - new TypeInformation[] {Types.LONG, Types.STRING}, - new String[] {"id", "name"})); - - Schema sourceSchema = - Schema.newBuilder() - .column("id", DataTypes.BIGINT()) - .column("name", DataTypes.STRING()) - .columnByExpression("proctime", "PROCTIME()") - .build(); - - tEnv.createTemporaryView("value_source", sourceStream, sourceSchema); - - if (caching == Caching.ENABLE_CACHE) { - LookupCacheManager.keepCacheOnRelease(true); - } - - // Execute lookup join - try (CloseableIterator iterator = - tEnv.executeSql( - "SELECT S.id, S.name, D._id, D.f1, D.f2 FROM value_source" - + " AS S JOIN mongo_source for system_time as of S.proctime AS D ON S.id = D._id") - .collect()) { - List result = - CollectionUtil.iteratorToList(iterator).stream() - .map(Row::toString) - .sorted() - .collect(Collectors.toList()); - List expected = - Arrays.asList( - "+I[1, Alice, 1, 2, false]", - "+I[1, Alice, 1, 2, false]", - "+I[2, Bob, 2, 2, false]"); - - assertThat(result).hasSize(3); - assertThat(result).isEqualTo(expected); - if (caching == Caching.ENABLE_CACHE) { - // Validate cache - Map managedCaches = - LookupCacheManager.getInstance().getManagedCaches(); - assertThat(managedCaches).hasSize(1); - LookupCache cache = - managedCaches.get(managedCaches.keySet().iterator().next()).getCache(); - validateCachedValues(cache); - } - - } finally { - if (caching == Caching.ENABLE_CACHE) { - LookupCacheManager.getInstance().checkAllReleased(); - LookupCacheManager.getInstance().clear(); - LookupCacheManager.keepCacheOnRelease(false); - } - } - } - - private static void validateCachedValues(LookupCache cache) { - // mongo does support project push down, the cached row has been projected - RowData key1 = GenericRowData.of(1L); - RowData value1 = GenericRowData.of(1L, StringData.fromString("2"), false); - - RowData key2 = GenericRowData.of(2L); - RowData value2 = GenericRowData.of(2L, StringData.fromString("2"), false); - - RowData key3 = GenericRowData.of(3L); - - Map> expectedEntries = new HashMap<>(); - expectedEntries.put(key1, Collections.singletonList(value1)); - expectedEntries.put(key2, Collections.singletonList(value2)); - expectedEntries.put(key3, Collections.emptyList()); - - LookupCacheAssert.assertThat(cache).containsExactlyEntriesOf(expectedEntries); - } - - private enum Caching { - ENABLE_CACHE, - DISABLE_CACHE - } - - private static String createTestDDl(Map extraOptions) { + private String createTestDDl(Map extraOptions) { Map options = new HashMap<>(); options.put(CONNECTOR.key(), "mongodb"); - options.put(URI.key(), MONGO_CONTAINER.getConnectionString()); + options.put(URI.key(), mongoDBContainer.getConnectionString()); options.put(DATABASE.key(), TEST_DATABASE); options.put(COLLECTION.key(), TEST_COLLECTION); if (extraOptions != null) { @@ -318,7 +301,7 @@ private static String createTestDDl(Map extraOptions) { Arrays.asList( "CREATE TABLE mongo_source", "(", - " _id BIGINT,", + " _id BIGINT PRIMARY KEY NOT ENFORCED,", " f1 STRING,", " f2 BOOLEAN,", " f3 BINARY,", @@ -361,4 +344,14 @@ private static BsonDocument createTestData(long id) { new BsonDocument("k", new BsonString("12_1")), new BsonDocument("k", new BsonString("12_2"))))); } + + private static List fetchRows(Iterator iter, int size) { + List rows = new ArrayList<>(size); + while (size > 0 && iter.hasNext()) { + Row row = iter.next(); + rows.add(row.toString()); + size--; + } + return rows; + } } diff --git a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoLookupTableSourceITCase.java b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoLookupTableSourceITCase.java new file mode 100644 index 00000000..1f7d1d92 --- /dev/null +++ b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoLookupTableSourceITCase.java @@ -0,0 +1,295 @@ +/* + * 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.flink.connector.mongodb.table; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.connector.mongodb.testutils.MongoTestUtil; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.connector.source.lookup.LookupOptions; +import org.apache.flink.table.connector.source.lookup.cache.LookupCache; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.runtime.functions.table.lookup.LookupCacheManager; +import org.apache.flink.table.test.lookup.cache.LookupCacheAssert; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.CollectionUtil; + +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoDatabase; +import com.mongodb.client.model.ChangeStreamPreAndPostImagesOptions; +import com.mongodb.client.model.CreateCollectionOptions; +import org.bson.BsonBinary; +import org.bson.BsonBoolean; +import org.bson.BsonDocument; +import org.bson.BsonInt64; +import org.bson.BsonString; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.MongoDBContainer; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.COLLECTION; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE; +import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.URI; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_4_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_5_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_6_0; +import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.MONGO_IMAGE_PREFIX; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for lookup feature of {@link MongoDynamicTableSource}. */ +@ExtendWith(ParameterizedTestExtension.class) +public class MongoLookupTableSourceITCase { + + private static final Logger LOG = LoggerFactory.getLogger(MongoDynamicTableSinkITCase.class); + + private static final String TEST_DATABASE = "test"; + private static final String TEST_COLLECTION = "mongo_lookup_source"; + + @RegisterExtension + static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .build()); + + @Parameter public String mongoVersion; + + @Parameter(value = 1) + public Caching caching; + + private static final List MONGO_VERSIONS = + Arrays.asList(MONGO_4_0, MONGO_5_0, MONGO_6_0); + + @Parameters(name = "mongoVersion={0} caching={1}") + public static List parameters() { + List params = new ArrayList<>(); + for (String version : MONGO_VERSIONS) { + params.add(new Object[] {version, Caching.DISABLE_CACHE}); + params.add(new Object[] {version, Caching.ENABLE_CACHE}); + } + return params; + } + + private MongoDBContainer mongoDBContainer; + private MongoClient mongoClient; + private MongoDatabase testDatabase; + private MongoCollection testCollection; + + private StreamExecutionEnvironment env; + private StreamTableEnvironment tEnv; + + @BeforeEach + void before() { + mongoDBContainer = + MongoTestUtil.createMongoDBContainer(MONGO_IMAGE_PREFIX + mongoVersion, LOG); + mongoDBContainer.start(); + mongoClient = MongoClients.create(mongoDBContainer.getConnectionString()); + testDatabase = mongoClient.getDatabase(TEST_DATABASE); + testCollection = + testDatabase.getCollection(TEST_COLLECTION).withDocumentClass(BsonDocument.class); + + if (MONGO_6_0.equals(mongoVersion)) { + testDatabase.createCollection( + TEST_COLLECTION, + new CreateCollectionOptions() + .changeStreamPreAndPostImagesOptions( + new ChangeStreamPreAndPostImagesOptions(true))); + } + + List testRecords = Arrays.asList(createTestData(1), createTestData(2)); + testCollection.insertMany(testRecords); + + env = StreamExecutionEnvironment.getExecutionEnvironment(); + tEnv = StreamTableEnvironment.create(env); + } + + @AfterEach + void after() { + if (mongoClient != null) { + mongoClient.close(); + } + mongoDBContainer.close(); + } + + @TestTemplate + public void testLookupJoin() throws Exception { + // Create MongoDB lookup table + Map lookupOptions = new HashMap<>(); + if (caching.equals(Caching.ENABLE_CACHE)) { + lookupOptions.put(LookupOptions.CACHE_TYPE.key(), "PARTIAL"); + lookupOptions.put(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE.key(), "10min"); + lookupOptions.put(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS.key(), "10min"); + lookupOptions.put(LookupOptions.PARTIAL_CACHE_MAX_ROWS.key(), "100"); + lookupOptions.put(LookupOptions.MAX_RETRIES.key(), "10"); + } + + tEnv.executeSql(createTestDDl(lookupOptions)); + + DataStream sourceStream = + env.fromCollection( + Arrays.asList( + Row.of(1L, "Alice"), + Row.of(1L, "Alice"), + Row.of(2L, "Bob"), + Row.of(3L, "Charlie"))) + .returns( + new RowTypeInfo( + new TypeInformation[] {Types.LONG, Types.STRING}, + new String[] {"id", "name"})); + + Schema sourceSchema = + Schema.newBuilder() + .column("id", DataTypes.BIGINT()) + .column("name", DataTypes.STRING()) + .columnByExpression("proctime", "PROCTIME()") + .build(); + + tEnv.createTemporaryView("value_source", sourceStream, sourceSchema); + + if (caching == Caching.ENABLE_CACHE) { + LookupCacheManager.keepCacheOnRelease(true); + } + + // Execute lookup join + try (CloseableIterator iterator = + tEnv.executeSql( + "SELECT S.id, S.name, D._id, D.f1, D.f2 FROM value_source" + + " AS S JOIN mongo_source for system_time as of S.proctime AS D ON S.id = D._id") + .collect()) { + List result = + CollectionUtil.iteratorToList(iterator).stream() + .map(Row::toString) + .sorted() + .collect(Collectors.toList()); + List expected = + Arrays.asList( + "+I[1, Alice, 1, 2, false]", + "+I[1, Alice, 1, 2, false]", + "+I[2, Bob, 2, 2, false]"); + + assertThat(result).hasSize(3); + assertThat(result).isEqualTo(expected); + if (caching == Caching.ENABLE_CACHE) { + // Validate cache + Map managedCaches = + LookupCacheManager.getInstance().getManagedCaches(); + assertThat(managedCaches).hasSize(1); + LookupCache cache = + managedCaches.get(managedCaches.keySet().iterator().next()).getCache(); + validateCachedValues(cache); + } + + } finally { + if (caching == Caching.ENABLE_CACHE) { + LookupCacheManager.getInstance().checkAllReleased(); + LookupCacheManager.getInstance().clear(); + LookupCacheManager.keepCacheOnRelease(false); + } + } + } + + private static void validateCachedValues(LookupCache cache) { + // mongo does support project push down, the cached row has been projected + RowData key1 = GenericRowData.of(1L); + RowData value1 = GenericRowData.of(1L, StringData.fromString("2"), false); + + RowData key2 = GenericRowData.of(2L); + RowData value2 = GenericRowData.of(2L, StringData.fromString("2"), false); + + RowData key3 = GenericRowData.of(3L); + + Map> expectedEntries = new HashMap<>(); + expectedEntries.put(key1, Collections.singletonList(value1)); + expectedEntries.put(key2, Collections.singletonList(value2)); + expectedEntries.put(key3, Collections.emptyList()); + + LookupCacheAssert.assertThat(cache).containsExactlyEntriesOf(expectedEntries); + } + + private enum Caching { + ENABLE_CACHE, + DISABLE_CACHE + } + + private String createTestDDl(Map extraOptions) { + Map options = new HashMap<>(); + options.put(CONNECTOR.key(), "mongodb"); + options.put(URI.key(), mongoDBContainer.getConnectionString()); + options.put(DATABASE.key(), TEST_DATABASE); + options.put(COLLECTION.key(), TEST_COLLECTION); + if (extraOptions != null) { + options.putAll(extraOptions); + } + + String optionString = + options.entrySet().stream() + .map(e -> String.format("'%s' = '%s'", e.getKey(), e.getValue())) + .collect(Collectors.joining(",\n")); + + return String.join( + "\n", + Arrays.asList( + "CREATE TABLE mongo_source", + "(", + " _id BIGINT PRIMARY KEY NOT ENFORCED,", + " f1 STRING,", + " f2 BOOLEAN,", + " f3 BINARY", + ") WITH (", + optionString, + ")")); + } + + private static BsonDocument createTestData(long id) { + return new BsonDocument() + .append("_id", new BsonInt64(id)) + .append("f1", new BsonString("2")) + .append("f2", BsonBoolean.FALSE) + .append("f3", new BsonBinary(new byte[] {(byte) 3})); + } +} diff --git a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/testutils/MongoShardedContainers.java b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/testutils/MongoShardedContainers.java index 3ea42e8f..09f7bbe3 100644 --- a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/testutils/MongoShardedContainers.java +++ b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/testutils/MongoShardedContainers.java @@ -52,25 +52,29 @@ public class MongoShardedContainers implements BeforeAllCallback, AfterAllCallba Slf4jLogConsumer logConsumer = new Slf4jLogConsumer(LOG); this.configSrv = new MongoDBContainer(dockerImageName) - .withCreateContainerCmdModifier(it -> it.withHostName(CONFIG_HOSTNAME)) - .withCommand( - "-configsvr", - "--replSet", - CONFIG_REPLICA_SET_NAME, - "--port", - String.valueOf(MONGODB_INTERNAL_PORT)) + .withCreateContainerCmdModifier( + it -> + it.withCmd( + "-configsvr", + "--replSet", + CONFIG_REPLICA_SET_NAME, + "--port", + String.valueOf(MONGODB_INTERNAL_PORT)) + .withHostName(CONFIG_HOSTNAME)) .withNetwork(network) .withNetworkAliases(CONFIG_HOSTNAME) .withLogConsumer(logConsumer); this.shardSrv = new MongoDBContainer(dockerImageName) - .withCreateContainerCmdModifier(it -> it.withHostName(SHARD_HOSTNAME)) - .withCommand( - "-shardsvr", - "--replSet", - SHARD_REPLICA_SET_NAME, - "--port", - String.valueOf(MONGODB_INTERNAL_PORT)) + .withCreateContainerCmdModifier( + it -> + it.withCmd( + "-shardsvr", + "--replSet", + SHARD_REPLICA_SET_NAME, + "--port", + String.valueOf(MONGODB_INTERNAL_PORT)) + .withHostName(SHARD_HOSTNAME)) .withNetwork(network) .withNetworkAliases(SHARD_HOSTNAME) .withLogConsumer(logConsumer); @@ -108,17 +112,21 @@ public String getConnectionString() { private static class MongoRouterContainer extends MongoDBContainer { private MongoRouterContainer(DockerImageName dockerImageName) { super(dockerImageName); - withCommand( - "mongos", - "--bind_ip_all", - "--configdb", - String.format( - "%s/%s:%d", - CONFIG_REPLICA_SET_NAME, CONFIG_HOSTNAME, MONGODB_INTERNAL_PORT)); + withCreateContainerCmdModifier( + it -> + it.withCmd( + "mongos", + "--bind_ip_all", + "--configdb", + String.format( + "%s/%s:%d", + CONFIG_REPLICA_SET_NAME, + CONFIG_HOSTNAME, + MONGODB_INTERNAL_PORT))); } @Override - protected void containerIsStarted(InspectContainerResponse containerInfo) { + protected void containerIsStarted(InspectContainerResponse containerInfo, boolean reused) { addShard(); } diff --git a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/testutils/MongoTestUtil.java b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/testutils/MongoTestUtil.java index 246f2bc9..4093f94a 100644 --- a/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/testutils/MongoTestUtil.java +++ b/flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/testutils/MongoTestUtil.java @@ -39,7 +39,10 @@ public class MongoTestUtil { public static final String MONGODB_HOSTNAME = "mongodb"; - public static final String MONGO_4_0 = "mongo:4.0.10"; + public static final String MONGO_IMAGE_PREFIX = "mongo:"; + public static final String MONGO_4_0 = "4.0.10"; + public static final String MONGO_5_0 = "5.0.2"; + public static final String MONGO_6_0 = "6.0.6"; private MongoTestUtil() {} @@ -50,7 +53,18 @@ private MongoTestUtil() {} * @return configured MongoDB container */ public static MongoDBContainer createMongoDBContainer(Logger logger) { - return new MongoDBContainer(DockerImageName.parse(MONGO_4_0)) + return createMongoDBContainer(MONGO_IMAGE_PREFIX + MONGO_4_0, logger); + } + + /** + * Creates a preconfigured {@link MongoDBContainer}. + * + * @param imageName mongo docker image name + * @param logger for test containers + * @return configured MongoDB container + */ + public static MongoDBContainer createMongoDBContainer(String imageName, Logger logger) { + return new MongoDBContainer(DockerImageName.parse(imageName)) .withLogConsumer(new Slf4jLogConsumer(logger)); } @@ -61,7 +75,19 @@ public static MongoDBContainer createMongoDBContainer(Logger logger) { * @return configured MongoDB sharded containers */ public static MongoShardedContainers createMongoDBShardedContainers(Network network) { - return new MongoShardedContainers(DockerImageName.parse(MONGO_4_0), network); + return createMongoDBShardedContainers(MONGO_IMAGE_PREFIX + MONGO_4_0, network); + } + + /** + * Creates a preconfigured {@link MongoDBContainer}. + * + * @param imageName mongo docker image name + * @param network for test containers + * @return configured MongoDB sharded containers + */ + public static MongoShardedContainers createMongoDBShardedContainers( + String imageName, Network network) { + return new MongoShardedContainers(DockerImageName.parse(imageName), network); } public static void assertThatIdsAreNotWritten(MongoCollection coll, Integer... ids) { diff --git a/pom.xml b/pom.xml index 431415a0..c3964c89 100644 --- a/pom.xml +++ b/pom.xml @@ -57,7 +57,7 @@ under the License. 5.8.1 3.21.0 - 1.17.2 + 1.18.3 3.4.6 false @@ -290,6 +290,13 @@ under the License. 2.24.0 + + + org.apache.commons + commons-compress + 1.23.0 + + org.objenesis