You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/10/11 11:32:31 UTC

[GitHub] [flink-connector-mongodb] Jiabao-Sun opened a new pull request, #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Jiabao-Sun opened a new pull request, #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1

   ## What is the purpose of the change
   
   FLIP-262 Flink MongoDB Connector 
   https://cwiki.apache.org/confluence/display/FLINK/FLIP-262%3A+Introduce+MongoDB+connector
   
   ## Features
   
     -  Support parallel read and write.
     -  Support lookup table source.
     -  Support scan table source.
     -  Support push limit down.
     -  Support push projection down.
   
   ## Documentation
   
   ### How to create a MongoDB table
   ```sql
   CREATE TABLE test_source (
     `_id` STRING,
     `idx` INT,
     `code` STRING,
     PRIMARY KEY (_id) NOT ENFORCED
   ) WITH (
     'connector' = 'mongodb',
     'uri' = 'mongodb://user:password@127.0.0.1:27017',
     'database' = 'test',
     'collection' = 'test_source'
   );
   
   CREATE TABLE test_sink (
     `_id` STRING,
     `idx` INT,
     `code` STRING,
     PRIMARY KEY (_id) NOT ENFORCED
   ) WITH (
     'connector' = 'mongodb',
     'uri' = 'mongodb://user:password@127.0.0.1:27017',
     'database' = 'test',
     'collection' = 'test_sink'
   );
   
   INSERT INTO test_sink SELECT * FROM test_source;
   ```
   
   ### How to create MongoDB Source
   ```java
     StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
   
     MongoSource<String> mongoSource =
            MongoSource.<String>builder()
                 .setUri("mongodb://user:password@127.0.0.1:27017")
                 .setDatabase("test")
                 .setCollection("test_source")
                 .setDeserializationSchema(new MongoJsonDeserializationSchema())
                 .build();
   
     env.fromSource(mongoSource, WatermarkStrategy.noWatermarks(), "MongoDB Source")
                 .setParallelism(4)
                 .print()
                 .setParallelism(1); 
   
     env.execute("Print MongoDB records");
   ```
   
   ### How to create MongoDB Sink
   ```java
     MongoSink<Document> sink =
            MongoSink.<Document>builder()
                 .setUri("mongodb://user:password@127.0.0.1:27017")
                 .setDatabase("test")
                 .setCollection("test_sink")
                 .setSerializationSchema((doc, ctx) -> new InsertOneModel<>(doc.toBsonDocument()))
                 .build();
   ```
   
   ### Connector Options
   
   
   Option | Required | Forwarded | Default | Type | Description
   -- | -- | -- | -- | -- | --
   connector | required |  no  | (none) | String | Specify what connector to use, here should be 'mongodb'.
   uri | required |  no  | (none) | String | Specifies the connection uri of MongoDB.
   database | required |  no  | (none) | String | Specifies the database to read or write of MongoDB.
   collection | required |  no  | (none) | String | Specifies the collection to read or write of MongoDB.
   scan.fetch-size | optional |  yes  |  2048 | Integer | Gives the reader a hint as to the number of documents that should be fetched from the database per round-trip when reading.
   scan.cursor.batch-size | optional |  yes  | 0 | Integer | Specifies the number of documents to return in each batch of the response from the MongoDB instance. Set to 0 to use server's default.
   scan.cursor.no-timeout | optional |  yes  | true | Boolean | The server normally times out idle cursors after an inactivity period (10 minutes) to prevent excess memory use. Set this option to true to prevent that.
   scan.partition.strategy | optional |  no  | 'default' | String | Specifies the partition strategy. Available strategies are single, sample, split-vector, sharded and default.
   scan.partition.size | optional |  no  | 64mb | MemorySize | Specifies the partition memory size.
   scan.partition.samples | optional |  no  | 10 | Integer | Specifies the the samples count per partition. It only takes effect when the partition strategy is sample.
   sink.bulk-flush.max-actions | optional |  yes  | 1000 | Integer | Specifies the maximum number of buffered actions per bulk request.
   sink.bulk-flush.interval | optional |  yes  | 1s | Integer | Specifies the bulk flush interval.
   sink.delivery-guarantee | optional |  no  | 'at-least-once' | String | Optional delivery guarantee when committing.
   sink.max-retries | optional |  yes  | 3 | Integer | Specifies the max retry times if writing records to database failed.
   sink.parallelism | optional |  no  | (none) | Integer |  Defines the parallelism of the MongoDB sink operator. By default, the parallelism is determined by the framework using the same parallelism of the upstream chained operator.
   lookup.max-retries | optional | yes	 | 3 |	Integer | The max retry times if lookup database failed.
   lookup.partial-cache.max-rows | optional | yes  | (none)| Integer | The maximum number of rows to store in the cache.
   lookup.partial-cache.cache-missing-key | optional | yes  | (none)| Boolean | Whether to store an empty value into the cache if the lookup key doesn't match any rows in the table.
   lookup.partial-cache.expire-after-access | optional | yes  | (none)| Duration | Duration to expire an entry in the cache after accessing.
   lookup.partial-cache.expire-after-write | optional | yes  | (none)| Duration | Duration to expire an entry in the cache after writing.
   
   
   ### DataType Mapping
   Bson Type | Flink SQL type 
   -- | -- 
   | BsonObjectId | STRING <br> CHAR <br>VARCHAR  |
   | BsonBoolean | BOOLEAN |
   | BsonBinary | BINARY <br> VARBINARY |
   | BsonInt32| TINYINT <br> SMALLINT <br> INT |
   | BsonInt64 | BIGINT |
   | BsonDouble | FLOAT <br> DOUBLE |
   | Decimal128 | DECIMAL |
   | BsonDateTime | TIMESTAMP_LTZ(3) |
   | BsonTimestamp | TIMESTAMP_LTZ(0) |
   | BsonString | STRING |
   | BsonSymbol | STRING |
   | BsonRegularExpression | STRING |
   | BsonJavaScript | STRING |
   | BsonDbPointer  | STRING <br> ROW<$ref STRING, $id STRING> |
   | BsonDocument | ROW |
   | BsonArray | ARRAY |
   | [GeoJson](https://www.mongodb.com/docs/manual/reference/geojson/) | Point : ROW<type STRING, coordinates ARRAY<DOUBLE>> <br> Line : ROW<type STRING, coordinates ARRAY<ARRAY< DOUBLE>>> <br>... |
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025484017


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable
+    public static BsonDocument readCollectionMetadata(
+            MongoClient mongoClient, MongoNamespace namespace) {
+        MongoCollection<BsonDocument> collection =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(COLLECTIONS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        return collection
+                .find(eq(ID_FIELD, namespace.getFullName()))
+                .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD))

Review Comment:
   > What is this `DROPPED_FIELD` about? Is this about whether the `collection` was dropped?
   
   Yes, this field exists in older versions of MongoDB. 
   
   Refers to https://jira.mongodb.org/browse/SERVER-27474
   >In releases up to and including 3.6, when a sharded collection was dropped, the config server, instead of deleting the dropped collection from its on-disk list of sharded collections, would instead tag it as "dropped". This was necessary because a <= 3.4 mongos, when a client tried to use the dropped collection, would get an error from the shard, and then would retrieve the whole collections list from the config server to update its collections cache. The mongos needs to see the collections with a "dropped" flag to know to discard its corresponding cached records of those collections and their chunks. As a consequence, other activities that read the list of collections have needed to filter out the "dropped" ones, and the records of dropped collections accumulate indefinitely.
   >
   >The 3.6 mongos no longer reads in the whole list of collections when it gets an "collection does not exist" error from a shard, but only the entry for the one collection, so does not need to see dropped collections identified. "Dropped" entries left over from <= 3.6 should be ignored/skipped when reading the collections list, and no new ones should be written by the 3.8 config server. Collection entries in 3.8 no longer need the "dropped" flag, and code that checks for it may be deleted.
   >
   >The upgrade process for 3.6 -> 3.8 config servers should scrub the remaining "dropped" entries from config.collections.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1031990672


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoJsonDeserializationSchema.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.reader.deserializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import org.bson.BsonDocument;
+import org.bson.json.JsonMode;
+
+import java.util.Optional;
+
+/**
+ * A schema bridge for deserializing the MongoDB's {@code BsonDocument} to MongoDB's {@link
+ * JsonMode}'s RELAXED Json string.
+ */
+@PublicEvolving
+public class MongoJsonDeserializationSchema implements MongoDeserializationSchema<String> {

Review Comment:
   Used to convert `BsonDocument` to [Json format](https://www.mongodb.com/docs/manual/reference/mongodb-extended-json/).
   
   ```java
       public void test() throws Exception {
           StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
           env.setParallelism(1);
   
           MongoSource<String> mongoSource =
                   MongoSource.<String>builder()
                           .setUri("mongodb://127.0.0.1:27017")
                           .setDatabase("test")
                           .setCollection("test")
                           .setDeserializationSchema(new MongoJsonDeserializationSchema())
                           .build();
   
           env.fromSource(mongoSource, WatermarkStrategy.noWatermarks(), "MongoSource")
                   .setParallelism(1)
                   .print();
   
           env.execute();
       }
   ```
   
   Output Example
   ```json
   {"_id": {"$oid": "63803fa3fdca9606df01da9e"}, "f0": 1, "f1": "hRKnchMIFNZMfAtBtYRViNleBezFgTSq"}
   {"_id": {"$oid": "63803fa3fdca9606df01da9f"}, "f0": 2, "f1": "oLRoMyLXvUxFwmOUINDwwhJeKpaSDJiM"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa0"}, "f0": 3, "f1": "esOnWaOwCUcsEGqisWNrupkbmIKtPZVS"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa1"}, "f0": 4, "f1": "lEqQYSNJKjUNILHFDnFQugpijBMXbqpy"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa2"}, "f0": 5, "f1": "lcgroaWbztgNIxUHwbVoVmSSLpmBYvXC"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa3"}, "f0": 6, "f1": "ELFXtzuJpLkWuIlXRVChRBqxnklbPGQF"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa4"}, "f0": 7, "f1": "WtFiNqcHCYmriyYFLBZtyqMPQDEidULJ"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa5"}, "f0": 8, "f1": "VDmoZYzuANQfpQtTnUdwXueNILPUgyNq"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa6"}, "f0": 9, "f1": "gLSwHUyzTHJjyOBrkjKlWTfsjtMeedef"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa7"}, "f0": 10, "f1": "hzHIcuDdvnEHcQYsKmPkbtmmzhWIyOTu"}
   ```
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032644671


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSink.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.MongoSink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataSerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A {@link DynamicTableSink} for MongoDB. */
+@Internal
+public class MongoDynamicTableSink implements DynamicTableSink {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    @Nullable private final Integer parallelism;
+    private final DataType physicalRowDataType;
+    private final SerializableFunction<RowData, BsonValue> keyExtractor;
+
+    public MongoDynamicTableSink(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            @Nullable Integer parallelism,
+            DataType physicalRowDataType,
+            SerializableFunction<RowData, BsonValue> keyExtractor) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.parallelism = parallelism;
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.keyExtractor = checkNotNull(keyExtractor);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        // UPSERT mode
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {

Review Comment:
   This connector can support writing in both append-only and upsert modes. I'm not sure if an explicitly upsert will force a primary key to be defined.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032644671


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSink.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.MongoSink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataSerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A {@link DynamicTableSink} for MongoDB. */
+@Internal
+public class MongoDynamicTableSink implements DynamicTableSink {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    @Nullable private final Integer parallelism;
+    private final DataType physicalRowDataType;
+    private final SerializableFunction<RowData, BsonValue> keyExtractor;
+
+    public MongoDynamicTableSink(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            @Nullable Integer parallelism,
+            DataType physicalRowDataType,
+            SerializableFunction<RowData, BsonValue> keyExtractor) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.parallelism = parallelism;
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.keyExtractor = checkNotNull(keyExtractor);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        // UPSERT mode
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {

Review Comment:
   ~~This connector can support writing in both append-only and upsert modes. I'm not sure if an explicitly upsert will force a primary key to be defined.~~



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] MartijnVisser commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r992214723


##########
pom.xml:
##########
@@ -0,0 +1,128 @@
+<?xml version="1.0" encoding="UTF-8"?>

Review Comment:
   You should model the POM like https://github.com/apache/flink-connector-elasticsearch/blob/main/pom.xml 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097315407


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.enumerator.splitter.MongoSplitters;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSplitAssigner implements MongoSplitAssigner {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSplitAssigner.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+
+    private final LinkedList<String> remainingCollections;
+    private final List<String> alreadyProcessedCollections;
+    private final List<MongoScanSourceSplit> remainingScanSplits;
+    private final Map<String, MongoScanSourceSplit> assignedScanSplits;
+    private boolean initialized;
+
+    private transient MongoSplitters mongoSplitters;
+
+    public MongoScanSplitAssigner(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            MongoSourceEnumState sourceEnumState) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.remainingCollections = new LinkedList<>(sourceEnumState.getRemainingCollections());

Review Comment:
   Got it, fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097287484


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriterITCase.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.connector.mongodb.testutils.MongoTestUtil;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.DeleteOneModel;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.UpdateOneModel;
+import com.mongodb.client.model.UpdateOptions;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+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.util.Optional;
+import java.util.OptionalLong;
+
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreNotWritten;
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link MongoWriter}. */
+@Testcontainers
+public class MongoWriterITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriterITCase.class);
+
+    private static final String TEST_DATABASE = "test_writer";
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    private static MongoClient mongoClient;
+    private static MetricListener metricListener;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void setUp() {
+        metricListener = new MetricListener();
+    }
+
+    @Test
+    void testWriteOnBulkFlush() throws Exception {
+        final String collection = "test-bulk-flush-without-checkpoint";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 5;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+
+            // Ignore flush on checkpoint
+            writer.flush(false);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3, 4);
+
+            // Trigger flush
+            writer.write(buildMessage(5), null);
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5);
+
+            writer.write(buildMessage(6), null);
+            assertThatIdsAreNotWritten(collectionOf(collection), 6);
+
+            // Force flush
+            writer.doBulkWrite();
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5, 6);
+        }
+    }
+
+    @Test
+    void testWriteOnBatchIntervalFlush() throws Exception {
+        final String collection = "test-bulk-flush-with-interval";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = -1;
+        final int batchIntervalMs = 1000;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+            writer.doBulkWrite();
+        }
+
+        assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4);
+    }
+
+    @Test
+    void testWriteOnCheckpoint() throws Exception {
+        final String collection = "test-bulk-flush-with-checkpoint";
+        final boolean flushOnCheckpoint = true;
+        final int batchSize = -1;
+        final int batchIntervalMs = -1;
+
+        // Enable flush on checkpoint
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3);
+
+            // Trigger flush
+            writer.flush(false);
+
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3);
+        }
+    }
+
+    @Test
+    void testIncrementRecordsSendMetric() throws Exception {
+        final String collection = "test-inc-records-send";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Counter> recordsSend =
+                    metricListener.getCounter(MetricNames.NUM_RECORDS_SEND);
+            writer.write(buildMessage(1), null);
+            // Update existing index
+            writer.write(buildMessage(2, "u"), null);
+            // Delete index
+            writer.write(buildMessage(3, "d"), null);
+
+            writer.doBulkWrite();
+
+            assertThat(recordsSend.isPresent()).isTrue();
+            assertThat(recordsSend.get().getCount()).isEqualTo(3L);
+        }
+    }
+
+    @Test
+    void testCurrentSendTime() throws Exception {
+        final String collection = "test-current-send-time";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Gauge<Long>> currentSendTime =
+                    metricListener.getGauge("currentSendTime");
+
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+
+            writer.doBulkWrite();
+
+            assertThat(currentSendTime.isPresent()).isTrue();
+            assertThat(currentSendTime.get().getValue()).isGreaterThan(0L);

Review Comment:
   >  There is a network IO operation, and under normal circumstances, it will be greater than 1ms.
   
   Since currentTimeMillis does not guarantee a monotonous sequence this isn't as simple :)
   
   
   As this test is only doing a single write there isn't really a better option (outside of retrying the entire test).
   If you'd be continuously writing data you could run a loop as long as `currentSendTime == 0`.
   
   It's just something that one should be aware of.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1052145855


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/converter/MongoConvertersTest.java:
##########
@@ -146,28 +146,31 @@ public void testConvertBsonToRowData() {
                 GenericRowData.of(
                         StringData.fromString(oid.toHexString()),
                         StringData.fromString("string"),
-                        StringData.fromString(uuid.toString()),
+                        StringData.fromString(
+                                "{\"_value\": {\"$binary\": {\"base64\": \"gR+qXamERr+L0IyxvO9daQ==\", \"subType\": \"04\"}}}"),
                         2,
                         3L,
                         4.1d,
                         DecimalData.fromBigDecimal(new BigDecimal("5.1"), 10, 2),
                         false,
                         TimestampData.fromEpochMillis(now.getEpochSecond() * 1000),
+                        TimestampData.fromEpochMillis(now.toEpochMilli()),
                         StringData.fromString(
-                                OffsetDateTime.ofInstant(
-                                                Instant.ofEpochMilli(now.toEpochMilli()),
-                                                ZoneOffset.UTC)
-                                        .format(ISO_OFFSET_DATE_TIME)),
-                        StringData.fromString("/^9$/i"),
-                        StringData.fromString("function() { return 10; }"),
-                        StringData.fromString("function() { return 11; }"),
-                        StringData.fromString("12"),
-                        StringData.fromString(oid.toHexString()),
+                                "{\"_value\": {\"$regularExpression\": {\"pattern\": \"^9$\", \"options\": \"i\"}}}"),

Review Comment:
   Thanks @twalthr.
   The `JsonWriter` cannot directly write a `BsonValue` in to a string. It will throw an exception when writing directly to a `BsonValue`, so we used `_value` to wrap the bson value into a bson document. However, we can also extend a `JsonWriter` so that it does not check when writing bson value directly.  Do you think we need to customize a `JsonWriter`?
   
   ```java
   package org.apache.flink;
   
   import org.bson.BsonRegularExpression;
   import org.bson.json.JsonWriter;
   
   import java.io.IOException;
   import java.io.StringWriter;
   
   public class JsonWriterTest {
   
       public static void main(String[] args) throws IOException {
           try (StringWriter stringWriter = new StringWriter();
                JsonWriter jsonWriter = new JsonWriter(stringWriter)) {
               BsonRegularExpression regularExpression = new BsonRegularExpression("regex", "i");
               jsonWriter.writeRegularExpression(regularExpression);
           }
       }
   }
   ```
   
   ```shell
   Exception in thread "main" org.bson.BsonInvalidOperationException: A RegularExpression value cannot be written to the root level of a BSON document.
   	at org.bson.AbstractBsonWriter.throwInvalidState(AbstractBsonWriter.java:740)
   	at org.bson.AbstractBsonWriter.checkPreconditions(AbstractBsonWriter.java:701)
   	at org.bson.AbstractBsonWriter.writeRegularExpression(AbstractBsonWriter.java:590)
   	at org.apache.flink.JsonWriterTest.main(JsonWriterTest.java:15)
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1018784724


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+    private static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+    static {
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME);

Review Comment:
   > We need some strategy to keep this up-to-date with newly added `LogicalTypeRoots` in Flink. Consider explicitly defining an allow/deny set covering the entire enum; that way you can detect newly added roots.
   
   DENIED types was defined to detected newly added roots.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011661631


##########
flink-connector-mongodb/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.architecture;
+
+import org.apache.flink.architecture.common.ImportOptions;
+
+import com.tngtech.archunit.core.importer.ImportOption;
+import com.tngtech.archunit.junit.AnalyzeClasses;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.junit.ArchTests;
+
+/** Architecture tests for test code. */
+@AnalyzeClasses(
+        packages = "org.apache.flink.connector.mongodb",
+        importOptions = {
+            ImportOption.OnlyIncludeTests.class,
+            ImportOptions.ExcludeScalaImportOption.class,
+            ImportOptions.ExcludeShadedImportOption.class
+        })
+public class TestCodeArchitectureTest {
+
+    @ArchTest
+    public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
+}

Review Comment:
   see https://github.com/apache/flink-connector-elasticsearch/blob/main/flink-connector-elasticsearch7/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java as an example



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010740286


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContextImpl;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible to write records in a MongoDB collection.
+ *
+ * @param <IN> The type of the input elements.
+ */
+@Internal
+public class MongoWriter<IN> implements SinkWriter<IN> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriter.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+    private final MongoSinkContext sinkContext;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean flushOnCheckpoint;
+    private final List<WriteModel<BsonDocument>> bulkRequests = new ArrayList<>();
+    private final Collector<WriteModel<BsonDocument>> collector;
+    private final MongoClient mongoClient;
+
+    private boolean checkpointInProgress = false;
+    private volatile long lastSendTime = 0L;
+    private volatile long ackTime = Long.MAX_VALUE;
+
+    public MongoWriter(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            boolean flushOnCheckpoint,
+            Sink.InitContext initContext,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);
+        this.flushOnCheckpoint = flushOnCheckpoint;
+
+        checkNotNull(initContext);
+        this.mailboxExecutor = checkNotNull(initContext.getMailboxExecutor());
+
+        SinkWriterMetricGroup metricGroup = checkNotNull(initContext.metricGroup());
+        metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+
+        this.collector =
+                new CountingCollector<>(
+                        new ListCollector<>(this.bulkRequests),
+                        metricGroup.getNumRecordsSendCounter());
+
+        // Initialize the serialization schema.
+        this.sinkContext = new MongoSinkContextImpl(initContext, writeOptions);
+        try {
+            SerializationSchema.InitializationContext initializationContext =
+                    initContext.asSerializationSchemaInitializationContext();
+            serializationSchema.open(initializationContext, sinkContext, writeOptions);
+        } catch (Exception e) {
+            throw new FlinkRuntimeException("Failed to open the MongoEmitter", e);
+        }
+
+        // Initialize the mongo client.
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    @Override
+    public void write(IN element, Context context) throws IOException, InterruptedException {
+        // do not allow new bulk writes until all actions are flushed
+        while (checkpointInProgress) {
+            mailboxExecutor.yield();
+        }
+        collector.collect(serializationSchema.serialize(element, sinkContext));
+        if (isOverMaxActionsLimit() || isOverMaxFlushIntervalLimit()) {
+            doBulkWrite();
+        }
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException {
+        checkpointInProgress = true;
+        while (!bulkRequests.isEmpty() && (flushOnCheckpoint || endOfInput)) {
+            doBulkWrite();
+        }
+        checkpointInProgress = false;
+    }
+
+    @Override
+    public void close() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @VisibleForTesting
+    void doBulkWrite() throws IOException {
+        if (bulkRequests.isEmpty()) {
+            // no records to write
+            return;
+        }
+
+        int maxRetryTimes = writeOptions.getMaxRetryTimes();
+        for (int i = 0; i <= maxRetryTimes; i++) {
+            try {
+                lastSendTime = System.currentTimeMillis();
+                mongoClient
+                        .getDatabase(connectionOptions.getDatabase())
+                        .getCollection(connectionOptions.getCollection(), BsonDocument.class)
+                        .bulkWrite(bulkRequests);
+                ackTime = System.currentTimeMillis();
+                bulkRequests.clear();
+                break;
+            } catch (MongoException e) {
+                LOG.error("Bulk Write to MongoDB failed, retry times = {}", i, e);

Review Comment:
   Changed the log level to debug for this line, and only log an error if it fails completely.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032653087


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoSourceSplitReader.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.reader.split;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import org.bson.BsonDocument;
+
+/**
+ * An split reader implements {@link SplitReader} for {@link MongoSourceSplit}.
+ *
+ * @param <T> Mongo source split.
+ */
+@Internal
+public interface MongoSourceSplitReader<T extends MongoSourceSplit>

Review Comment:
   Reserved for the future implementation of continuous reading through [change stream](https://www.mongodb.com/docs/manual/changeStreams/#change-streams) feature.
   
   - MongoSourceSplitReader (interface):`MongoScanSourceSplitReader` and `MongoStreamSourceSplitReader`
   - MongoSourceSplit (abstract): `MongoScanSplit` and `MongoStreamSplit`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1030397682


##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,117 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!--using mongodb shade jar to execute end-to-end test-->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-sql-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-deploy-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>

Review Comment:
   This needs to be setup similarly to https://github.com/apache/flink-connector-elasticsearch/blob/main/flink-connector-elasticsearch-e2e-tests/pom.xml#L44; a dedicated profile that activates the test.
   (This is because these tests need some prep work beforehand, specifically downloading a Flink binary)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1031737091


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = ID_FIELD;
+
+    private static final AppendOnlyKeyExtractor APPEND_ONLY_KEY_EXTRACTOR =
+            new AppendOnlyKeyExtractor();
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reservedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // Primary key is not declared and reserved _id is not present.

Review Comment:
   This situation we directly set _id as `null` and let Mongo Server generate a unique `ObjectId`.
   The `ObjectId` depends on the server time and increment.
   We did not generate it on the client side because it may not be accurate.
   If there is a better way, please give some suggestions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032971595


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoShardedSplitter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.mongodb.MongoException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isShardedCollectionDropped;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readChunks;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readCollectionMetadata;
+
+/**
+ * Sharded Partitioner
+ *
+ * <p>Uses the chunks collection and partitions the collection based on the sharded collections
+ * chunk ranges.
+ *
+ * <p>The following config collections' read privilege is required.
+ *
+ * <ul>
+ *   <li>config.collections
+ *   <li>config.chunks
+ * </ul>
+ */
+@Internal
+public class MongoShardedSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+    public static final MongoShardedSplitter INSTANCE = new MongoShardedSplitter();
+
+    private MongoShardedSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoClient mongoClient = splitContext.getMongoClient();
+
+        List<BsonDocument> chunks;
+        Optional<BsonDocument> collectionMetadata;
+        try {
+            collectionMetadata = readCollectionMetadata(mongoClient, namespace);
+            if (!collectionMetadata.isPresent()) {
+                LOG.error(
+                        "Do sharded split failed, collection {} does not appear to be sharded.",
+                        namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, %s is not a sharded collection.",
+                                namespace));
+            }
+
+            if (isShardedCollectionDropped(collectionMetadata.get())) {
+                LOG.error("Do sharded split failed, collection {} was dropped.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format("Do sharded split failed, %s was dropped.", namespace));
+            }
+
+            chunks = readChunks(mongoClient, collectionMetadata.get());
+            if (chunks.isEmpty()) {
+                LOG.error("Do sharded split failed, chunks of {} is empty.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, chunks of %s is empty.", namespace));
+            }
+        } catch (MongoException e) {
+            LOG.error(
+                    "Read chunks from {} failed with error message: {}", namespace, e.getMessage());
+            throw new FlinkRuntimeException(e);
+        }
+
+        List<MongoScanSourceSplit> sourceSplits = new ArrayList<>(chunks.size());
+        for (int i = 0; i < chunks.size(); i++) {
+            BsonDocument chunk = chunks.get(i);
+            sourceSplits.add(
+                    new MongoScanSourceSplit(
+                            String.format("%s_%d", namespace, i),

Review Comment:
   How about we use the primary key (`_id` field mentioned above) of `config.chunks`?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032989565


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSink.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.MongoSink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataSerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A {@link DynamicTableSink} for MongoDB. */
+@Internal
+public class MongoDynamicTableSink implements DynamicTableSink {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    @Nullable private final Integer parallelism;
+    private final DataType physicalRowDataType;
+    private final SerializableFunction<RowData, BsonValue> keyExtractor;
+
+    public MongoDynamicTableSink(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            @Nullable Integer parallelism,
+            DataType physicalRowDataType,
+            SerializableFunction<RowData, BsonValue> keyExtractor) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.parallelism = parallelism;
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.keyExtractor = checkNotNull(keyExtractor);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        // UPSERT mode
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {

Review Comment:
   We have added tests for these two scenarios in E2E.
   Explicitly set  an upsert `ChangelogMode` won't force a primary key to be defined.
   So change the `ChangelogMode` to upsert mode here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#issuecomment-1278556930

   Hi @zentol.
   Looking forward to your feedback.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017645688


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializer.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+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.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeList;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeMap;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeList;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeMap;
+
+/** The {@link SimpleVersionedSerializer Serializer} for the enumerator state of Mongo source. */
+@Internal
+public class MongoSourceEnumStateSerializer
+        implements SimpleVersionedSerializer<MongoSourceEnumState> {
+
+    public static final MongoSourceEnumStateSerializer INSTANCE =
+            new MongoSourceEnumStateSerializer();
+
+    private MongoSourceEnumStateSerializer() {
+        // Singleton instance.
+    }
+
+    @Override
+    public int getVersion() {
+        // We use MongoSourceSplitSerializer's version because we use reuse this class.
+        return MongoSourceSplitSerializer.CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(MongoSourceEnumState state) throws IOException {
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                DataOutputStream out = new DataOutputStream(baos)) {
+            serializeList(out, state.getRemainingCollections(), DataOutputStream::writeUTF);
+
+            serializeList(out, state.getAlreadyProcessedCollections(), DataOutputStream::writeUTF);
+
+            serializeList(
+                    out,
+                    state.getRemainingScanSplits(),
+                    MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
+
+            serializeMap(
+                    out,
+                    state.getAssignedScanSplits(),
+                    DataOutputStream::writeUTF,
+                    MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
+
+            out.writeBoolean(state.isInitialized());
+
+            out.flush();
+            return baos.toByteArray();
+        }
+    }
+
+    @Override
+    public MongoSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
+        try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
+                DataInputStream in = new DataInputStream(bais)) {
+            List<String> remainingCollections = deserializeList(in, DataInput::readUTF);
+            List<String> alreadyProcessedCollections = deserializeList(in, DataInput::readUTF);
+            List<MongoScanSourceSplit> remainingScanSplits =
+                    deserializeList(in, i -> deserializeMongoScanSourceSplit(version, i));
+
+            Map<String, MongoScanSourceSplit> assignedScanSplits =
+                    deserializeMap(
+                            in,
+                            DataInput::readUTF,
+                            i -> deserializeMongoScanSourceSplit(version, i));
+
+            boolean initialized = in.readBoolean();
+
+            return new MongoSourceEnumState(
+                    remainingCollections,
+                    alreadyProcessedCollections,
+                    remainingScanSplits,
+                    assignedScanSplits,
+                    initialized);
+        }
+    }
+
+    private MongoScanSourceSplit deserializeMongoScanSourceSplit(int version, DataInputStream in)
+            throws IOException {
+        return (MongoScanSourceSplit)

Review Comment:
   MongoDB can support not only batch reads, but also streaming reads.
   We can use MongoDB [change streams](https://www.mongodb.com/docs/manual/changeStreams/#change-streams) feature to read operation changes, and then convert them into flink's upsert mode changelog.
   
   Batch read is currently implemented in this PR, some abstractions are also left for streaming reading:
   MongoSourceSplit -> MongoScanSourceSplit
   MongoSourceSplit -> MongoStreamSourceSplit
   
   Here I made some adjustments in recent commit, check the split type to avoid type casting.
   Whether we need to left some abstractions for this now, I would like to hear your opinion.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017019902


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.split.MongoScanSourceSplitReader;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The configuration class for MongoDB source. */
+@PublicEvolving
+public class MongoReadOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int fetchSize;
+
+    private final int cursorBatchSize;
+
+    private final boolean noCursorTimeout;
+
+    private final PartitionStrategy partitionStrategy;
+
+    private final MemorySize partitionSize;
+
+    private final int samplesPerPartition;
+
+    private MongoReadOptions(
+            int fetchSize,
+            int cursorBatchSize,
+            boolean noCursorTimeout,
+            PartitionStrategy partitionStrategy,
+            MemorySize partitionSize,
+            int samplesPerPartition) {
+        this.fetchSize = fetchSize;
+        this.cursorBatchSize = cursorBatchSize;
+        this.noCursorTimeout = noCursorTimeout;
+        this.partitionStrategy = partitionStrategy;
+        this.partitionSize = partitionSize;
+        this.samplesPerPartition = samplesPerPartition;
+    }
+
+    public int getFetchSize() {
+        return fetchSize;
+    }
+
+    public int getCursorBatchSize() {
+        return cursorBatchSize;
+    }
+
+    public boolean isNoCursorTimeout() {
+        return noCursorTimeout;
+    }
+
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    public MemorySize getPartitionSize() {
+        return partitionSize;
+    }
+
+    public int getSamplesPerPartition() {
+        return samplesPerPartition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoReadOptions that = (MongoReadOptions) o;
+        return cursorBatchSize == that.cursorBatchSize
+                && noCursorTimeout == that.noCursorTimeout
+                && partitionStrategy == that.partitionStrategy
+                && samplesPerPartition == that.samplesPerPartition
+                && Objects.equals(partitionSize, that.partitionSize);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                cursorBatchSize,
+                noCursorTimeout,
+                partitionStrategy,
+                partitionSize,
+                samplesPerPartition);
+    }
+
+    public static MongoReadOptionsBuilder builder() {
+        return new MongoReadOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoReadOptions}. */
+    @PublicEvolving
+    public static class MongoReadOptionsBuilder {
+        private int fetchSize = SCAN_FETCH_SIZE.defaultValue();
+        private int cursorBatchSize = SCAN_CURSOR_BATCH_SIZE.defaultValue();
+        private boolean noCursorTimeout = SCAN_CURSOR_NO_TIMEOUT.defaultValue();
+        private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue();
+        private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue();
+        private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue();
+
+        /**
+         * Sets the number of documents should be fetched per round-trip when reading.
+         *
+         * @param fetchSize the number of documents should be fetched per round-trip when reading.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setFetchSize(int fetchSize) {
+            checkArgument(fetchSize > 0, "The fetch size must be larger than 0.");
+            this.fetchSize = fetchSize;
+            return this;
+        }
+
+        /**
+         * Sets the batch size of MongoDB find cursor.
+         *
+         * @param cursorBatchSize the max batch size of find cursor.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setCursorBatchSize(int cursorBatchSize) {
+            checkArgument(
+                    cursorBatchSize >= 0,
+                    "The cursor batch size must be larger than or equal to 0.");
+            this.cursorBatchSize = cursorBatchSize;
+            return this;
+        }
+
+        /**
+         * Set this option to true to prevent cursor timeout (10 minutes).

Review Comment:
   The MongoDB server normally times out idle cursors after an inactivity period (10minutes) to prevent excess memory use. 
   
   Refers to https://www.mongodb.com/docs/manual/reference/method/cursor.noCursorTimeout/#session-idle-timeout-overrides-nocursortimeout
   ```
   If a session is idle for longer than 30 minutes, the MongoDB server marks that session as expired 
   and may close it at any time. 
   When the MongoDB server closes the session, it also kills any in-progress operations and open 
   cursors associated with the session. 
   This includes cursors configured with noCursorTimeout() or a maxTimeMS() greater than 30 minutes.
   ```
   
   We can use this option to make the cursor live longer than 10 minutes, but it will be less than 30 minutes.
   This is useful for some scenarios where the query takes a long time.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1018113562


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.DefaultMongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible for writing records to a MongoDB collection.
+ *
+ * @param <IN> The type of the input elements.
+ */
+@Internal
+public class MongoWriter<IN> implements SinkWriter<IN> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriter.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+    private final MongoSinkContext sinkContext;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean flushOnCheckpoint;
+    private final List<WriteModel<BsonDocument>> bulkRequests = new ArrayList<>();
+    private final Collector<WriteModel<BsonDocument>> collector;
+    private final MongoClient mongoClient;
+
+    private boolean checkpointInProgress = false;
+    private volatile long lastSendTime = 0L;
+    private volatile long ackTime = Long.MAX_VALUE;
+
+    public MongoWriter(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            boolean flushOnCheckpoint,
+            Sink.InitContext initContext,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);
+        this.flushOnCheckpoint = flushOnCheckpoint;
+
+        checkNotNull(initContext);
+        this.mailboxExecutor = checkNotNull(initContext.getMailboxExecutor());
+
+        SinkWriterMetricGroup metricGroup = checkNotNull(initContext.metricGroup());
+        metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+
+        this.collector =
+                new CountingCollector<>(
+                        new ListCollector<>(this.bulkRequests),
+                        metricGroup.getNumRecordsSendCounter());
+
+        // Initialize the serialization schema.
+        this.sinkContext = new DefaultMongoSinkContext(initContext, writeOptions);
+        try {
+            SerializationSchema.InitializationContext initializationContext =
+                    initContext.asSerializationSchemaInitializationContext();
+            serializationSchema.open(initializationContext, sinkContext, writeOptions);
+        } catch (Exception e) {
+            throw new FlinkRuntimeException("Failed to open the MongoEmitter", e);
+        }
+
+        // Initialize the mongo client.
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    @Override
+    public void write(IN element, Context context) throws IOException, InterruptedException {
+        // do not allow new bulk writes until all actions are flushed
+        while (checkpointInProgress) {
+            mailboxExecutor.yield();
+        }
+        collector.collect(serializationSchema.serialize(element, sinkContext));
+        if (isOverMaxBatchSizeLimit() || isOverMaxBatchIntervalLimit()) {
+            doBulkWrite();
+        }
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException {
+        checkpointInProgress = true;
+        while (!bulkRequests.isEmpty() && (flushOnCheckpoint || endOfInput)) {
+            doBulkWrite();
+        }
+        checkpointInProgress = false;
+    }
+
+    @Override
+    public void close() {
+        mongoClient.close();
+    }
+
+    @VisibleForTesting
+    void doBulkWrite() throws IOException {
+        if (bulkRequests.isEmpty()) {
+            // no records to write
+            return;
+        }
+
+        int maxRetries = writeOptions.getMaxRetries();

Review Comment:
   Thanks @rozza 
   
   When using Flink SQL, we will use the primary key of the Flink table schema as the _id of MongoDB.
   Then process upsert or delete operations by that key. I think it's idempotent in this case.
   
   When using Flink Streaming, how to convert record to `BsonDocument` and  how to write to MongoDB will be decided by user.  If the write method are non-idempotent, some problem may occur on retries.
   
   Thanks to suggest retryable writes feature that seems' to be able to give us a higher delivery guarantee.
   I'll try that.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1019100632


##########
flink-sql-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,95 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-sql-connector-mongodb</artifactId>
+	<name>Flink : Connectors : SQL : MongoDB</name>
+
+	<packaging>jar</packaging>
+
+	<properties>
+		<japicmp.skip>true</japicmp.skip>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet>
+								<includes>
+									<include>org.apache.flink:flink-connector-base</include>
+									<include>org.apache.flink:flink-connector-mongodb</include>
+									<include>org.mongodb:bson</include>
+									<include>org.mongodb:mongodb-driver-sync</include>
+									<include>org.mongodb:mongodb-driver-core</include>
+								</includes>
+							</artifactSet>
+							<filters>
+								<filter>
+									<artifact>org.mongodb:mongodb-driver-core</artifact>
+									<excludes>
+										<exclude>META-INF/native-image/**/**.properties</exclude>

Review Comment:
   The native-image was used for GraalVM, I think we don't need it.
   But I found that using maven shade plugin doesn't seem exclude it, is there something wrong ?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1044117642


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.annotation.Internal;
+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.MongoReadOptions;
+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;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
+import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** A {@link DynamicTableSource} for MongoDB. */
+@Internal
+public class MongoDynamicTableSource
+        implements ScanTableSource,
+                LookupTableSource,
+                SupportsProjectionPushDown,
+                SupportsLimitPushDown {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    @Nullable private final LookupCache lookupCache;
+    private final int lookupMaxRetries;
+    private final long lookupRetryIntervalMs;
+    private DataType physicalRowDataType;
+    private int limit = -1;
+
+    public MongoDynamicTableSource(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            @Nullable LookupCache lookupCache,
+            int lookupMaxRetries,
+            long lookupRetryIntervalMs,
+            DataType physicalRowDataType) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.lookupCache = lookupCache;
+        checkArgument(
+                lookupMaxRetries >= 0,
+                String.format(
+                        "The '%s' must be larger than or equal to 0.",
+                        LookupOptions.MAX_RETRIES.key()));
+        checkArgument(
+                lookupRetryIntervalMs > 0,
+                String.format("The '%s' must be larger than 0.", LOOKUP_RETRY_INTERVAL.key()));
+        this.lookupMaxRetries = lookupMaxRetries;
+        this.lookupRetryIntervalMs = lookupRetryIntervalMs;
+        this.physicalRowDataType = physicalRowDataType;
+    }
+
+    @Override
+    public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
+        final List<String> keyNames = new ArrayList<>(context.getKeys().length);
+        for (int[] innerKeyArr : context.getKeys()) {
+            Preconditions.checkArgument(
+                    innerKeyArr.length == 1, "MongoDB only support non-nested look up keys yet");
+            keyNames.add(DataType.getFieldNames(physicalRowDataType).get(innerKeyArr[0]));
+        }
+        final RowType rowType = (RowType) physicalRowDataType.getLogicalType();
+
+        MongoRowDataLookupFunction lookupFunction =
+                new MongoRowDataLookupFunction(
+                        connectionOptions,
+                        lookupMaxRetries,
+                        lookupRetryIntervalMs,
+                        DataType.getFieldNames(physicalRowDataType),
+                        DataType.getFieldDataTypes(physicalRowDataType),
+                        keyNames,
+                        rowType);
+        if (lookupCache != null) {
+            return PartialCachingLookupProvider.of(lookupFunction, lookupCache);
+        } else {
+            return LookupFunctionProvider.of(lookupFunction);
+        }
+    }
+
+    @Override
+    public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
+        final RowType rowType = (RowType) physicalRowDataType.getLogicalType();
+        final TypeInformation<RowData> typeInfo =
+                runtimeProviderContext.createTypeInformation(physicalRowDataType);
+
+        final MongoDeserializationSchema<RowData> deserializationSchema =
+                new MongoRowDataDeserializationSchema(rowType, typeInfo);
+
+        MongoSource<RowData> mongoSource =
+                MongoSource.<RowData>builder()
+                        .setUri(connectionOptions.getUri())
+                        .setDatabase(connectionOptions.getDatabase())
+                        .setCollection(connectionOptions.getCollection())
+                        .setFetchSize(readOptions.getFetchSize())
+                        .setCursorBatchSize(readOptions.getCursorBatchSize())
+                        .setNoCursorTimeout(readOptions.isNoCursorTimeout())
+                        .setPartitionStrategy(readOptions.getPartitionStrategy())
+                        .setPartitionSize(readOptions.getPartitionSize())
+                        .setSamplesPerPartition(readOptions.getSamplesPerPartition())
+                        .setLimit(limit)
+                        .setProjectedFields(DataType.getFieldNames(physicalRowDataType))
+                        .setDeserializationSchema(deserializationSchema)
+                        .build();
+
+        return SourceProvider.of(mongoSource);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode() {
+        return ChangelogMode.insertOnly();
+    }
+
+    @Override
+    public DynamicTableSource copy() {
+        return new MongoDynamicTableSource(
+                connectionOptions,
+                readOptions,
+                lookupCache,
+                lookupMaxRetries,
+                lookupRetryIntervalMs,
+                physicalRowDataType);
+    }
+
+    @Override
+    public String asSummaryString() {
+        return "MongoDB";
+    }
+
+    @Override
+    public void applyLimit(long limit) {
+        this.limit = (int) limit;
+    }
+
+    @Override
+    public boolean supportsNestedProjection() {
+        // planner doesn't support nested projection push down yet.
+        return false;
+    }
+
+    @Override
+    public void applyProjection(int[][] projectedFields, DataType producedDataType) {
+        this.physicalRowDataType = Projection.of(projectedFields).project(physicalRowDataType);

Review Comment:
   Thanks, fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097361784


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriterITCase.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.connector.mongodb.testutils.MongoTestUtil;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.DeleteOneModel;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.UpdateOneModel;
+import com.mongodb.client.model.UpdateOptions;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+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.util.Optional;
+import java.util.OptionalLong;
+
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreNotWritten;
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link MongoWriter}. */
+@Testcontainers
+public class MongoWriterITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriterITCase.class);
+
+    private static final String TEST_DATABASE = "test_writer";
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    private static MongoClient mongoClient;
+    private static MetricListener metricListener;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void setUp() {
+        metricListener = new MetricListener();
+    }
+
+    @Test
+    void testWriteOnBulkFlush() throws Exception {
+        final String collection = "test-bulk-flush-without-checkpoint";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 5;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+
+            // Ignore flush on checkpoint
+            writer.flush(false);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3, 4);
+
+            // Trigger flush
+            writer.write(buildMessage(5), null);
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5);
+
+            writer.write(buildMessage(6), null);
+            assertThatIdsAreNotWritten(collectionOf(collection), 6);
+
+            // Force flush
+            writer.doBulkWrite();
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5, 6);
+        }
+    }
+
+    @Test
+    void testWriteOnBatchIntervalFlush() throws Exception {
+        final String collection = "test-bulk-flush-with-interval";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = -1;
+        final int batchIntervalMs = 1000;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+            writer.doBulkWrite();
+        }
+
+        assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4);
+    }
+
+    @Test
+    void testWriteOnCheckpoint() throws Exception {
+        final String collection = "test-bulk-flush-with-checkpoint";
+        final boolean flushOnCheckpoint = true;
+        final int batchSize = -1;
+        final int batchIntervalMs = -1;
+
+        // Enable flush on checkpoint
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3);
+
+            // Trigger flush
+            writer.flush(false);
+
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3);
+        }
+    }
+
+    @Test
+    void testIncrementRecordsSendMetric() throws Exception {
+        final String collection = "test-inc-records-send";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Counter> recordsSend =
+                    metricListener.getCounter(MetricNames.NUM_RECORDS_SEND);
+            writer.write(buildMessage(1), null);
+            // Update existing index
+            writer.write(buildMessage(2, "u"), null);
+            // Delete index
+            writer.write(buildMessage(3, "d"), null);
+
+            writer.doBulkWrite();
+
+            assertThat(recordsSend.isPresent()).isTrue();
+            assertThat(recordsSend.get().getCount()).isEqualTo(3L);
+        }
+    }
+
+    @Test
+    void testCurrentSendTime() throws Exception {
+        final String collection = "test-current-send-time";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Gauge<Long>> currentSendTime =
+                    metricListener.getGauge("currentSendTime");
+
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+
+            writer.doBulkWrite();
+
+            assertThat(currentSendTime.isPresent()).isTrue();
+            assertThat(currentSendTime.get().getValue()).isGreaterThan(0L);

Review Comment:
   Thanks @zentol  for reminding me of this problem. 
   Retry mechanism has been added into this test case to make it more stable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097492206


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSampleSplitter.class);
+
+    public static final MongoSampleSplitter INSTANCE = new MongoSampleSplitter();
+
+    private MongoSampleSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+
+        long count = splitContext.getCount();
+        long partitionSizeInBytes = readOptions.getPartitionSize().getBytes();
+        int samplesPerPartition = readOptions.getSamplesPerPartition();
+
+        long avgObjSizeInBytes = splitContext.getAvgObjSize();
+        if (avgObjSizeInBytes == 0L) {
+            LOG.info(
+                    "{} seems to be an empty collection, Returning a single partition.", namespace);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        long numDocumentsPerPartition = partitionSizeInBytes / avgObjSizeInBytes;
+        if (numDocumentsPerPartition >= count) {
+            LOG.info(
+                    "Fewer documents ({}) than the number of documents per partition ({}), Returning a single partition.",
+                    count,
+                    numDocumentsPerPartition);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        int numberOfSamples =
+                (int) Math.ceil((samplesPerPartition * count * 1.0d) / numDocumentsPerPartition);

Review Comment:
   ```suggestion
           int numberOfPartitions = (int) Math.ceil(count * 1.0d / numDocumentsPerPartition);
           int numberOfSamples = samplesPerPartition * numbersOfPartitions;
   ```
   This made it easier to understand for me.
   



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSampleSplitter.class);
+
+    public static final MongoSampleSplitter INSTANCE = new MongoSampleSplitter();
+
+    private MongoSampleSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+
+        long count = splitContext.getCount();

Review Comment:
   ```suggestion
           long totalNumDocuments = splitContext.getCount();
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter {

Review Comment:
   I'd love to see a test for this splitter. It's currently too easy to do a refactoring that changes the output by accident.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSampleSplitter.class);
+
+    public static final MongoSampleSplitter INSTANCE = new MongoSampleSplitter();
+
+    private MongoSampleSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+
+        long count = splitContext.getCount();
+        long partitionSizeInBytes = readOptions.getPartitionSize().getBytes();
+        int samplesPerPartition = readOptions.getSamplesPerPartition();
+
+        long avgObjSizeInBytes = splitContext.getAvgObjSize();
+        if (avgObjSizeInBytes == 0L) {
+            LOG.info(
+                    "{} seems to be an empty collection, Returning a single partition.", namespace);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        long numDocumentsPerPartition = partitionSizeInBytes / avgObjSizeInBytes;
+        if (numDocumentsPerPartition >= count) {
+            LOG.info(
+                    "Fewer documents ({}) than the number of documents per partition ({}), Returning a single partition.",
+                    count,
+                    numDocumentsPerPartition);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        int numberOfSamples =
+                (int) Math.ceil((samplesPerPartition * count * 1.0d) / numDocumentsPerPartition);
+
+        List<BsonDocument> samples =
+                splitContext
+                        .getMongoCollection()
+                        .aggregate(
+                                Arrays.asList(
+                                        Aggregates.sample(numberOfSamples),
+                                        Aggregates.project(Projections.include(ID_FIELD)),
+                                        Aggregates.sort(Sorts.ascending(ID_FIELD))))
+                        .allowDiskUse(true)
+                        .into(new ArrayList<>());
+
+        List<MongoScanSourceSplit> sourceSplits = new ArrayList<>();
+        BsonDocument partitionStart = new BsonDocument(ID_FIELD, BSON_MIN_KEY);
+        int splitNum = 0;
+        for (int i = 0; i < samples.size(); i++) {
+            if (i % samplesPerPartition == 0 || i == samples.size() - 1) {
+                sourceSplits.add(
+                        createSplit(namespace, splitNum++, partitionStart, samples.get(i)));
+                partitionStart = samples.get(i);
+            }
+        }

Review Comment:
   ```suggestion
           BsonDocument partitionStart = samples.get(0);
           int splitNum = 0;
           for (int i = samplesPerPartition - 1; i < samples.size(); i+=samplesPerPartition) {
                   sourceSplits.add(
                           createSplit(namespace, splitNum++, partitionStart, samples.get(i)));
                   partitionStart = samples.get(i);
           }
   ```
   
   Ideally this is how the loop would look like. We just have a list of points and split it into `numPartition` blocks. No need for special handling of MIN/MAX or to explicitly complete the right bound.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1098879627


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter {

Review Comment:
   Sorry for the late reply.
   Unit test `MongoSampleSplitterTest` has been added for `MongoSampleSplitter`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097671216


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter {

Review Comment:
   Currently we can test this splitter by `MongoSourceITCase.testPartitionStrategy` case. 
   Further testing of this splitter will be done tomorrow.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1044082086


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue() == 1;
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue() == 1L;
+        }
+        if (bsonValue.isString()) {
+            return Boolean.parseBoolean(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to boolean from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static byte convertToTinyInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (byte) 1 : (byte) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (byte) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (byte) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Byte.parseByte(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to tinyint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static short convertToSmallInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (short) 1 : (short) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (short) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (short) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Short.parseShort(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to smallint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static int convertToInt(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().intValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.intValue();
+            } else if (decimal128Value.isNegative()) {
+                return Integer.MIN_VALUE;
+            } else {
+                return Integer.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isDateTime()) {
+            return (int) Instant.ofEpochMilli(bsonValue.asDateTime().getValue()).getEpochSecond();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime();
+        }
+        if (bsonValue.isString()) {
+            return Integer.parseInt(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to integer from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static long convertToLong(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().longValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.longValue();
+            } else if (decimal128Value.isNegative()) {
+                return Long.MIN_VALUE;
+            } else {
+                return Long.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1L : 0L;
+        }
+        if (bsonValue.isDateTime()) {
+            return bsonValue.asDateTime().getValue();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime() * 1000L;
+        }
+        if (bsonValue.isString()) {
+            return Long.parseLong(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to long from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static double convertToDouble(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().doubleValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.doubleValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Double.MAX_VALUE;
+            } else {
+                return Double.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isString()) {
+            return Double.parseDouble(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to double from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static float convertToFloat(BsonValue bsonValue) {
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {

Review Comment:
   MongoDB uses the [BSON](https://www.mongodb.com/json-and-bson) structure internally, and the extend-json format is a way to convert BSON into JSON.  
   We can get the BSON raw string-based value in the following way.
   
   ```java
   import org.bson.BsonDocument;
   import org.bson.BsonRegularExpression;
   import org.bson.BsonSymbol;
   
   public class BsonTest {
   
       public static void main(String[] args) {
           BsonSymbol symbol = new BsonSymbol("symbol");
           BsonRegularExpression regex = new BsonRegularExpression("regex", "i");
   
           BsonDocument doc = new BsonDocument()
                   .append("f0", symbol)
                   .append("f1", regex);
   
           // convert doc to json
           String extendJson = doc.toJson();
   
           // output: {"f0": {"$symbol": "symbol"}, "f1": {"$regularExpression": {"pattern": "regex", "options": "i"}}}
           System.out.println(extendJson);
   
           // restore from json string
           BsonDocument parsed = BsonDocument.parse(extendJson);
       }
   }
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1016634420


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom

Review Comment:
   Those are not _custom_ options though.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1016664140


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017559683


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumState.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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;
+
+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.split.MongoScanSourceSplit;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The state class for MongoDB source enumerator, used for storing the split state. This class is
+ * managed and controlled by {@link MongoSplitAssigner}.
+ */
+@PublicEvolving
+public class MongoSourceEnumState {
+
+    /** The Mongo collections remaining. */
+    private final List<String> remainingCollections;
+
+    /**
+     * The paths that are no longer in the enumerator checkpoint, but have been processed before.
+     */
+    private final List<String> alreadyProcessedCollections;

Review Comment:
   No, it doesn't.
   
   The `remainingCollections` stores the MongoDB collections (like table name) we need to read.
   The `alreadyProcessedCollections` stores the collections we have already read.
   These collection names are limited and can be determined at startup.
   
   Currently, we explicitly qualify database and collection, and store at most one collection name in it.
   Designing the state as a list is to maintain state compatibility for future support for multi-collection queries.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1012083456


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom

Review Comment:
   Sorry, I missed this comment.
   We can set these options by `MongoSinkBuilder`.
   Here's an example:
   ```java
   MongoSink<String> sink =
                   MongoSink.<String>builder()
                           .setUri("mongodb://127.0.0.1:27017")
                           .setDatabase("test")
                           .setCollection("test")
                           .setBulkFlushMaxActions(5)
                           .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
                           .setSerializationSchema(new MongoSerializationSchema<String>() {
                               @Override
                               public void open(
                                       SerializationSchema.InitializationContext initializationContext,
                                       MongoSinkContext sinkContext,
                                       MongoWriteOptions sinkConfiguration) throws Exception {
                                   MongoSerializationSchema.super.open(
                                           initializationContext,
                                           sinkContext,
                                           sinkConfiguration);
                                   // we can do something here
                               }
   
                               @Override
                               public WriteModel<BsonDocument> serialize(
                                       String element,
                                       MongoSinkContext sinkContext) {
                                   return new InsertOneModel<>(BsonDocument.parse(element));
                               }
                           })
                           .build();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1030553397


##########
flink-sql-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-sql-connector-mongodb</artifactId>
+	<name>Flink : Connectors : SQL : MongoDB</name>
+
+	<packaging>jar</packaging>
+
+	<properties>
+		<japicmp.skip>true</japicmp.skip>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet>
+								<includes>
+									<include>org.apache.flink:flink-connector-mongodb</include>

Review Comment:
   To ease maintenance I'd set this to `*:*` to include everything the connector needs.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializerTest.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonMaxKey;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+import static org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumStateSerializer.INSTANCE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+
+/** Unit tests for {@link MongoSourceEnumStateSerializer}. */
+public class MongoSourceEnumStateSerializerTest {
+
+    @Test
+    void serializeAndDeserializeMongoSourceEnumState() throws Exception {
+        boolean initialized = false;
+        List<String> remainingCollections = Arrays.asList("db.remains0", "db.remains1");
+        List<String> alreadyProcessedCollections = Arrays.asList("db.processed0", "db.processed1");
+        List<MongoScanSourceSplit> remainingScanSplits =
+                Arrays.asList(createSourceSplit(0), createSourceSplit(1));
+
+        Map<String, MongoScanSourceSplit> assignedScanSplits =
+                Collections.singletonMap("split2", createSourceSplit(2));
+
+        MongoSourceEnumState state =
+                new MongoSourceEnumState(
+                        remainingCollections,
+                        alreadyProcessedCollections,
+                        remainingScanSplits,
+                        assignedScanSplits,
+                        initialized);
+
+        byte[] bytes = INSTANCE.serialize(state);
+        MongoSourceEnumState state1 = INSTANCE.deserialize(INSTANCE.getVersion(), bytes);
+
+        assertEquals(state.getRemainingCollections(), state1.getRemainingCollections());
+        assertEquals(
+                state.getAlreadyProcessedCollections(), state1.getAlreadyProcessedCollections());
+        assertEquals(state.getRemainingScanSplits(), state1.getRemainingScanSplits());
+        assertEquals(state.getAssignedScanSplits(), state1.getAssignedScanSplits());
+        assertEquals(state.isInitialized(), state1.isInitialized());
+
+        assertNotSame(state, state1);
+    }
+
+    private MongoScanSourceSplit createSourceSplit(int index) {

Review Comment:
   ```suggestion
       private static MongoScanSourceSplit createSourceSplit(int index) {
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/common/utils/MongoSerdeUtilsTest.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.junit.jupiter.api.Test;
+
+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.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Unit tests for {@link MongoSerdeUtils}. */
+public class MongoSerdeUtilsTest {

Review Comment:
   ```suggestion
   class MongoSerdeUtilsTest {
   ```
   Test classes/methods no longer need to be public.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = ID_FIELD;
+
+    private static final AppendOnlyKeyExtractor APPEND_ONLY_KEY_EXTRACTOR =
+            new AppendOnlyKeyExtractor();
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {

Review Comment:
   May nee changes depending on other discussions about supported types.
   Just marking this here as a reminder.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/MongoConfiguration.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+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.COLLECTION;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.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;
+
+/** MongoDB configuration. */
+@PublicEvolving
+public class MongoConfiguration implements Serializable {

Review Comment:
   Does not have to be serializable afaict.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = ID_FIELD;
+
+    private static final AppendOnlyKeyExtractor APPEND_ONLY_KEY_EXTRACTOR =
+            new AppendOnlyKeyExtractor();
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reservedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // Primary key is not declared and reserved _id is not present.

Review Comment:
   Curious whether other connectors have any logic to silently use a field as a key. Sounds a bit iffy to me. :shrug: 
   Seems that the ES connector would generate a random id in this case (because of how the client works).



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/PartitionStrategy.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+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;
+
+/** MongoSplitStrategy that can be chosen. */
+@PublicEvolving
+public enum PartitionStrategy implements DescribedEnum {

Review Comment:
   Is this something that mongodb users are expected to know? Maybe we should document these.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/common/utils/MongoSerdeUtilsTest.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.junit.jupiter.api.Test;
+
+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.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Unit tests for {@link MongoSerdeUtils}. */
+public class MongoSerdeUtilsTest {
+
+    @Test
+    public void testSerializeList() throws IOException {
+        List<String> expected = Arrays.asList("config.collections", "config.chunks");
+
+        byte[] serialized = serializeList(expected);
+        List<String> deserialized = deserializeList(serialized);
+
+        assertThat(deserialized).isEqualTo(expected);
+    }
+
+    @Test
+    public void testSerializeMap() throws IOException {
+        Map<String, String> expected = new HashMap<>();
+        expected.put("k0", "v0");
+        expected.put("k1", "v1");
+        expected.put("k2", "v2");
+
+        byte[] serialized = serializeMap(expected);
+
+        Map<String, String> deserialized = deserializeMap(serialized);
+
+        assertThat(deserialized).isEqualTo(expected);
+    }
+
+    private byte[] serializeList(List<String> list) throws IOException {

Review Comment:
   these methods should be static



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializerTest.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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;
+
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonMaxKey;
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+import static org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumStateSerializer.INSTANCE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+
+/** Unit tests for {@link MongoSourceEnumStateSerializer}. */
+public class MongoSourceEnumStateSerializerTest {
+
+    @Test
+    void serializeAndDeserializeMongoSourceEnumState() throws Exception {
+        boolean initialized = false;
+        List<String> remainingCollections = Arrays.asList("db.remains0", "db.remains1");
+        List<String> alreadyProcessedCollections = Arrays.asList("db.processed0", "db.processed1");
+        List<MongoScanSourceSplit> remainingScanSplits =
+                Arrays.asList(createSourceSplit(0), createSourceSplit(1));
+
+        Map<String, MongoScanSourceSplit> assignedScanSplits =
+                Collections.singletonMap("split2", createSourceSplit(2));
+
+        MongoSourceEnumState state =
+                new MongoSourceEnumState(
+                        remainingCollections,
+                        alreadyProcessedCollections,
+                        remainingScanSplits,
+                        assignedScanSplits,
+                        initialized);
+
+        byte[] bytes = INSTANCE.serialize(state);
+        MongoSourceEnumState state1 = INSTANCE.deserialize(INSTANCE.getVersion(), bytes);
+
+        assertEquals(state.getRemainingCollections(), state1.getRemainingCollections());
+        assertEquals(
+                state.getAlreadyProcessedCollections(), state1.getAlreadyProcessedCollections());
+        assertEquals(state.getRemainingScanSplits(), state1.getRemainingScanSplits());
+        assertEquals(state.getAssignedScanSplits(), state1.getAssignedScanSplits());
+        assertEquals(state.isInitialized(), state1.isInitialized());
+
+        assertNotSame(state, state1);
+    }
+
+    private MongoScanSourceSplit createSourceSplit(int index) {
+        return new MongoScanSourceSplit(
+                "split" + index,
+                "db",
+                "coll",
+                new BsonDocument("_id", new BsonInt32(index)),
+                new BsonDocument("_id", new BsonMaxKey()),

Review Comment:
   ```suggestion
                   new BsonDocument("_id", MongoConstants.BSON_MAX_KEY),
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = ID_FIELD;
+
+    private static final AppendOnlyKeyExtractor APPEND_ONLY_KEY_EXTRACTOR =
+            new AppendOnlyKeyExtractor();
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);

Review Comment:
   Seems inefficient to recreate this getter for each value. The `FieldGetter` is supposed to be serializable, so it should be possible to do this once in the constructor (or initialize it lazily otherwise).



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Serializable, Closeable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
+
+    private final MongoReadOptions readOptions;
+    private final boolean limitPushedDown;
+    private final MongoClient mongoClient;
+
+    public MongoSplitters(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            boolean limitPushedDown) {
+        this.readOptions = readOptions;
+        this.limitPushedDown = limitPushedDown;
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    public Collection<MongoScanSourceSplit> split(MongoNamespace namespace) {
+        BsonDocument collStats = MongoUtils.collStats(mongoClient, namespace);
+        if (!isCommandSucceed(collStats)) {
+            LOG.error(
+                    "Execute command collStats failed: {}",
+                    collStats.getString(ERROR_MESSAGE_FIELD));
+            throw new IllegalStateException(String.format("Collection not found %s", namespace));
+        }
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(readOptions, mongoClient, namespace, collStats);
+
+        if (limitPushedDown) {
+            LOG.info("Limit {} is applied, using single splitter", limitPushedDown);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }

Review Comment:
   According to the `SupportsLimitPushDown` javadocs the limiting can be done on a best-effort basis. So for every split we could emit up to N elements, and have the Table runtime deal with the rest.
   



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/MongoConfiguration.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.MemorySize;
+import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+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.COLLECTION;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.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;
+
+/** MongoDB configuration. */
+@PublicEvolving

Review Comment:
   ```suggestion
   @Internal
   ```
   This isn't user-facing is it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1030587220


##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,156 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Use fat jar so we don't need to create a user-jar. -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-sql-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<profiles>
+		<profile>
+			<id>run-end-to-end-tests</id>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<executions>
+							<execution>
+								<id>end-to-end-tests</id>
+								<phase>integration-test</phase>
+								<goals>
+									<goal>test</goal>
+								</goals>
+								<configuration>
+									<includes>
+										<include>**/*.*</include>
+									</includes>
+									<forkCount>1</forkCount>

Review Comment:
   Sorry, didn't figure this out before. It has been fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1028782035


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+@ExtendWith(TestLoggerExtension.class)
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);

Review Comment:
   Hi @zentol,
   
   The [MongoDB test container](https://www.testcontainers.org/modules/databases/mongodb/) only supports [replica set](https://www.mongodb.com/docs/manual/replication/#replication-in-mongodb) mode, while the sharded split strategy requires MongoDB to run in [sharded cluster](https://www.mongodb.com/docs/manual/sharding/#sharded-cluster) mode.  So here mocks some result of [config.collections](https://www.mongodb.com/docs/manual/reference/config-database/#mongodb-data-config.collections) and [config.chunks](https://www.mongodb.com/docs/manual/reference/config-database/#mongodb-data-config.chunks)  to simulate the scenario of sharded cluster.
   
   Also, I'm a bit confused about these functional wrappers, using them we might still need to mock their results. Is there something wrong with my understanding?
   
   BTW, We can also do some extra work to start 3 mongo containers and have them run in sharded mode. Do we need to take this approach?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032977743


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.source.enumerator.MongoSourceEnumState;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoSourceSplit}. */
+@Internal
+public interface MongoSplitAssigner extends Serializable {
+
+    /**
+     * Called to open the assigner to acquire any resources, like threads or network connections.
+     */
+    void open();
+
+    /**
+     * Called to close the assigner, in case it holds on to any resources, like threads or network
+     * connections.
+     */
+    void close() throws IOException;
+
+    /** Gets the next split. */
+    Optional<MongoSourceSplit> getNext();

Review Comment:
   When all splits have been assigned, but the enumerator has not signaled no more splits yet.
   When the enumerator receives an empty split, it will check whether it should notify the reader to close.
   
   `MongoSourceEnumerator#assignSplits`
   ```java
   Optional<MongoSourceSplit> split = splitAssigner.getNext();
   if (split.isPresent()) {
       final MongoSourceSplit mongoSplit = split.get();
       context.assignSplit(mongoSplit, nextAwaiting);
       awaitingReader.remove();
       LOG.info("Assign split {} to subtask {}", mongoSplit, nextAwaiting);
       break;
   } else if (splitAssigner.noMoreSplits() && boundedness == Boundedness.BOUNDED) {
       LOG.info("All splits have been assigned");
       context.registeredReaders().keySet().forEach(context::signalNoMoreSplits);
       break;
   } else {
       // there is no available splits by now, skip assigning
       break;
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025533564


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable

Review Comment:
   The [config.collections] stores a document for each sharded (only sharded) collection in the cluster.
   If the collection is not sharded then returns null.
   I will use `Optional` instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] twalthr commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
twalthr commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1051933589


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/converter/MongoConvertersTest.java:
##########
@@ -146,28 +146,31 @@ public void testConvertBsonToRowData() {
                 GenericRowData.of(
                         StringData.fromString(oid.toHexString()),
                         StringData.fromString("string"),
-                        StringData.fromString(uuid.toString()),
+                        StringData.fromString(
+                                "{\"_value\": {\"$binary\": {\"base64\": \"gR+qXamERr+L0IyxvO9daQ==\", \"subType\": \"04\"}}}"),
                         2,
                         3L,
                         4.1d,
                         DecimalData.fromBigDecimal(new BigDecimal("5.1"), 10, 2),
                         false,
                         TimestampData.fromEpochMillis(now.getEpochSecond() * 1000),
+                        TimestampData.fromEpochMillis(now.toEpochMilli()),
                         StringData.fromString(
-                                OffsetDateTime.ofInstant(
-                                                Instant.ofEpochMilli(now.toEpochMilli()),
-                                                ZoneOffset.UTC)
-                                        .format(ISO_OFFSET_DATE_TIME)),
-                        StringData.fromString("/^9$/i"),
-                        StringData.fromString("function() { return 10; }"),
-                        StringData.fromString("function() { return 11; }"),
-                        StringData.fromString("12"),
-                        StringData.fromString(oid.toHexString()),
+                                "{\"_value\": {\"$regularExpression\": {\"pattern\": \"^9$\", \"options\": \"i\"}}}"),

Review Comment:
   shouldn't the string be just `{\"$regularExpression\": {\"pattern\": \"^9$\", \"options\": \"i\"}}` without the outer `_value`? or is this important for the round trip?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1044071447


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSourceITCase.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.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 org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.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.BeforeEach;
+import org.junit.jupiter.api.Test;
+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.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+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.table.factories.FactoryUtil.CONNECTOR;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase for {@link MongoDynamicTableSource}. */
+@Testcontainers
+public class MongoDynamicTableSourceITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoDynamicTableSinkITCase.class);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    public static final String TEST_DATABASE = "test";
+    public static final String TEST_COLLECTION = "mongo_table_source";
+
+    private static MongoClient mongoClient;
+
+    public static StreamExecutionEnvironment env;
+    public static StreamTableEnvironment tEnv;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+
+        MongoCollection<BsonDocument> coll =
+                mongoClient
+                        .getDatabase(TEST_DATABASE)
+                        .getCollection(TEST_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        List<BsonDocument> testRecords = Arrays.asList(createTestData(1), createTestData(2));
+        coll.insertMany(testRecords);
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void before() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+    }
+
+    @Test
+    public void testSource() {
+        tEnv.executeSql(createTestDDl(null));
+
+        Iterator<Row> collected = tEnv.executeSql("SELECT * FROM mongo_source").collect();
+        List<String> result =
+                CollectionUtil.iteratorToList(collected).stream()
+                        .map(Row::toString)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        List<String> expected =
+                Stream.of(
+                                "+I[1, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]",
+                                "+I[2, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]")
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        assertThat(result).isEqualTo(expected);
+    }
+
+    @Test
+    public void testProject() {
+        tEnv.executeSql(createTestDDl(null));
+
+        Iterator<Row> collected = tEnv.executeSql("SELECT f1, f13 FROM mongo_source").collect();
+        List<String> result =
+                CollectionUtil.iteratorToList(collected).stream()
+                        .map(Row::toString)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        List<String> expected =
+                Stream.of("+I[2, +I[13]]", "+I[2, +I[13]]").sorted().collect(Collectors.toList());
+
+        assertThat(result).isEqualTo(expected);
+    }
+
+    @Test
+    public void testLimit() {
+        tEnv.executeSql(createTestDDl(null));
+
+        Iterator<Row> collected = tEnv.executeSql("SELECT * FROM mongo_source LIMIT 1").collect();
+        List<String> result =
+                CollectionUtil.iteratorToList(collected).stream()
+                        .map(Row::toString)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        Set<String> expected = new HashSet<>();
+        expected.add(
+                "+I[1, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]");
+        expected.add(
+                "+I[2, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]");
+
+        assertThat(result).hasSize(1);
+        assertThat(result).containsAnyElementsOf(expected);
+    }
+
+    @ParameterizedTest
+    @EnumSource(Caching.class)
+    public void testLookupJoin(Caching caching) throws Exception {
+        // Create MongoDB lookup table
+        Map<String, String> 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<Row> 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<Row> 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<String> result =
+                    CollectionUtil.iteratorToList(iterator).stream()
+                            .map(Row::toString)
+                            .sorted()
+                            .collect(Collectors.toList());
+            List<String> 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<String, LookupCacheManager.RefCountedCache> 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<RowData, Collection<RowData>> 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<String, String> extraOptions) {
+        Map<String, String> options = new HashMap<>();
+        options.put(CONNECTOR.key(), "mongodb");
+        options.put(URI.key(), MONGO_CONTAINER.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,",

Review Comment:
   The MongoDB's `_id` is similar to Elasticsearch's `_id`. 
   
   The field name _id is reserved for use as a primary key;  Its value must be unique in the collection, is immutable, and may be of any type other than an array. 
   
   https://www.mongodb.com/docs/manual/core/document/#the-_id-field
   > In MongoDB, each document stored in a collection requires a unique [_id](https://www.mongodb.com/docs/manual/reference/glossary/#std-term-_id) field that acts as a [primary key](https://www.mongodb.com/docs/manual/reference/glossary/#std-term-primary-key). If an inserted document omits the _id field, the MongoDB driver automatically generates an [ObjectId](https://www.mongodb.com/docs/manual/reference/bson-types/#std-label-objectid) for the _id field.
   >
   > The _id field has the following behavior and constraints:
   > - By default, MongoDB creates a unique index on the _id field during the creation of a collection.
   > - The _id field may contain values of any [BSON data type](https://www.mongodb.com/docs/manual/reference/bson-types/), other than an array, regex, or undefined.
   > - _ If the _id contains subfields, the subfield names cannot begin with a ($) symbol.
   
   The round-trip semantics is already supported with this PR.
   The content of the sink can be re-read by the source ideally with exactly the same schema.
   However, since some mongodb-specific types cannot be directly mapped to Flink sql types, when converting `BsonSymbol`, `BsonRegularExpression`, `BsonJavaScript`, `BsonDbPointer` to String, we may lose the original type information.
   In other words, reading from one collection of MongoDB and writing to another collection of MongoDB, the schema of the two collections of may be different. Suppose we map `BsonSymbol` to Flink sql `String`, when we write string to MongoDB again, we are not sure whether it is mapped to `BsonSymbol` or `BsonString`.
   
   But if we convert mongodb-specific types into extend-json format, we can keep the original types, restore the original types accurately, and let users use UDF to process them.
   
   https://www.mongodb.com/docs/manual/reference/mongodb-extended-json/
   
   
   
   
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1044175579


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,677 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());

Review Comment:
   Thanks @twalthr to catch this.
   Both `RowDataToBsonConverters` and `BsonToRowDataConverters` are fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097291736


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.enumerator.splitter.MongoSplitters;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSplitAssigner implements MongoSplitAssigner {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSplitAssigner.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+
+    private final LinkedList<String> remainingCollections;
+    private final List<String> alreadyProcessedCollections;
+    private final List<MongoScanSourceSplit> remainingScanSplits;
+    private final Map<String, MongoScanSourceSplit> assignedScanSplits;
+    private boolean initialized;
+
+    private transient MongoSplitters mongoSplitters;
+
+    public MongoScanSplitAssigner(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            MongoSourceEnumState sourceEnumState) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.remainingCollections = new LinkedList<>(sourceEnumState.getRemainingCollections());

Review Comment:
   >  Do we need to explicitly declare it as a Queue?
   
   No; my main point is that we use 2 different types of lists although we want similar semantics.
   
   Basically: Why is `remainingScanSplits` not also a LinkedList?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097524812


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.

Review Comment:
   OK; now I finally got what this splitter is doing. :+1: 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1095898919


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/RowDataToBsonConverters.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonNull;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.json.JsonParseException;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+
+/** Tool class used to convert from {@link RowData} to {@link BsonValue}. */
+@Internal
+public class RowDataToBsonConverters {
+
+    // --------------------------------------------------------------------------------
+    // Runtime Converters
+    // --------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts objects of Flink Table & SQL internal data structures to
+     * corresponding {@link BsonValue} data structures.
+     */
+    @FunctionalInterface
+    public interface RowDataToBsonConverter extends Serializable {
+        BsonValue convert(Object value);
+    }

Review Comment:
   This interface is for the conversion of specific sql types later, and the parameter may not be clearly defined as `SqlData`. 
   For example, `DecimalData` and `TimestampData` do not have a specific parent class like `SqlData`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032955663


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoScanSourceSplitReader.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.api.connector.source.SourceReaderContext;
+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.MongoReadOptions;
+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 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;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.project;
+
+/** An split reader implements {@link SplitReader} for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSourceSplitReader implements MongoSourceSplitReader<MongoSourceSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSourceSplitReader.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    private final SourceReaderContext readerContext;
+    @Nullable private final List<String> projectedFields;
+    private final int limit;
+
+    private boolean closed = false;
+    private boolean finished = false;
+    private MongoClient mongoClient;
+    private MongoCursor<BsonDocument> currentCursor;
+    private MongoScanSourceSplit currentSplit;
+
+    public MongoScanSourceSplitReader(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            @Nullable List<String> projectedFields,
+            int limit,
+            SourceReaderContext context) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.projectedFields = projectedFields;
+        this.limit = limit;
+        this.readerContext = context;
+    }
+
+    @Override
+    public RecordsWithSplitIds<BsonDocument> fetch() throws IOException {
+        if (closed) {
+            throw new IllegalStateException("Cannot fetch records from a closed split reader");
+        }
+
+        RecordsBySplits.Builder<BsonDocument> builder = new RecordsBySplits.Builder<>();
+
+        // Return when no split registered to this reader.
+        if (currentSplit == null) {
+            return builder.build();
+        }
+
+        currentCursor = getOrCreateCursor();
+        int fetchSize = readOptions.getFetchSize();
+
+        try {
+            for (int recordNum = 0; recordNum < fetchSize; recordNum++) {
+                if (currentCursor.hasNext()) {
+                    builder.add(currentSplit, currentCursor.next());
+                } else {
+                    builder.addFinishedSplit(currentSplit.splitId());
+                    finished = true;
+                    break;
+                }
+            }

Review Comment:
   We use a [cursor](https://www.mongodb.com/docs/manual/reference/method/cursor.batchSize/#mongodb-method-cursor.batchSize) to request a batch of data from mongodb, the size of the batch depends on the configuration of `scan.cursor.batch-size`.  No request will be made to mongodb until a batch of data in the cursor has been processed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032983150


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoScanSourceSplitReader.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.api.connector.source.SourceReaderContext;
+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.MongoReadOptions;
+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 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;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.project;
+
+/** An split reader implements {@link SplitReader} for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSourceSplitReader implements MongoSourceSplitReader<MongoSourceSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSourceSplitReader.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    private final SourceReaderContext readerContext;
+    @Nullable private final List<String> projectedFields;
+    private final int limit;
+
+    private boolean closed = false;
+    private boolean finished = false;
+    private MongoClient mongoClient;
+    private MongoCursor<BsonDocument> currentCursor;
+    private MongoScanSourceSplit currentSplit;
+
+    public MongoScanSourceSplitReader(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            @Nullable List<String> projectedFields,
+            int limit,
+            SourceReaderContext context) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.projectedFields = projectedFields;
+        this.limit = limit;
+        this.readerContext = context;
+    }
+
+    @Override
+    public RecordsWithSplitIds<BsonDocument> fetch() throws IOException {
+        if (closed) {
+            throw new IllegalStateException("Cannot fetch records from a closed split reader");
+        }
+
+        RecordsBySplits.Builder<BsonDocument> builder = new RecordsBySplits.Builder<>();
+
+        // Return when no split registered to this reader.
+        if (currentSplit == null) {
+            return builder.build();
+        }
+
+        currentCursor = getOrCreateCursor();
+        int fetchSize = readOptions.getFetchSize();
+
+        try {
+            for (int recordNum = 0; recordNum < fetchSize; recordNum++) {
+                if (currentCursor.hasNext()) {
+                    builder.add(currentSplit, currentCursor.next());
+                } else {
+                    builder.addFinishedSplit(currentSplit.splitId());
+                    finished = true;
+                    break;
+                }
+            }
+            return builder.build();
+        } catch (MongoException e) {
+            throw new IOException("Scan records form MongoDB failed", e);
+        } finally {
+            if (finished) {
+                currentSplit = null;
+                releaseCursor();
+            }
+        }
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<MongoSourceSplit> 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()));
+        }
+
+        MongoSourceSplit sourceSplit = splitsChanges.splits().get(0);
+        if (!(sourceSplit instanceof MongoScanSourceSplit)) {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "The SourceSplit type of %s is not supported.",
+                            sourceSplit.getClass()));
+        }
+
+        this.currentSplit = (MongoScanSourceSplit) sourceSplit;
+        this.finished = false;
+    }
+
+    @Override
+    public void wakeUp() {}

Review Comment:
   We took the approach of closing the cursor  to cancel blocked hasNext(), next() when the wakeUp method is invoked.
   
   `MongoCursor#colse`
   > Despite this interface being non-thread-safe, close() is allowed to be called concurrently with any method of the cursor, including itself. This is useful to cancel blocked hasNext(), next(). This method is idempotent.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032994738


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.

Review Comment:
   > If multiple samples are taken per partition then somewhere in here we'd have to merge sample to arrive at a single partition again, but afaict that doesn't happen.
   
   We merge samples in the following code.
   ```java
           List<MongoScanSourceSplit> sourceSplits = new ArrayList<>();
           BsonDocument partitionStart = new BsonDocument(ID_FIELD, BSON_MIN_KEY);
           int splitNum = 0;
           for (int i = 0; i < samples.size(); i++) {
               if (i % samplesPerPartition == 0 || i == samples.size() - 1) {
                   sourceSplits.add(
                           createSplit(namespace, splitNum++, partitionStart, samples.get(i)));
                   partitionStart = samples.get(i);
               }
           }
   ```
   
   ____________
   
   > Instead we have some strange formula that determines the number of samples (read: partitions), and I have no idea how the resulting partitions could correlate with the desired partition size.
   >
   > Why isnt the number of sample (again: partitions) not count / numDocumentsPerPartition?
   
   1. numDocumentsPerPartition = partitionSizeInBytes / avgObjSizeInBytes
   2. samplingRate = samplesPerPartition / numDocumentsPerPartition
   3. samplesCount = samplingRate * count
   4. merge samples by samplesPerPartition
   
   We calculate the sampling rate through samples per partition and partition size. We can also be accomplished directly by setting the sampling rate. 
   
   @zentol `scan.partition.samples` or `scan.partition.sampling-rate` which do you think is better?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032635261


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.time.Duration;
+
+/**
+ * Base options for the MongoDB connector. Needs to be public so that the {@link
+ * org.apache.flink.table.api.TableDescriptor} can access it.
+ */
+@PublicEvolving
+public class MongoConnectorOptions {
+
+    private MongoConnectorOptions() {}
+
+    public static final ConfigOption<String> URI =
+            ConfigOptions.key("uri")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the connection uri of MongoDB.");
+
+    public static final ConfigOption<String> DATABASE =
+            ConfigOptions.key("database")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the database to read or write of MongoDB.");
+
+    public static final ConfigOption<String> COLLECTION =
+            ConfigOptions.key("collection")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the collection to read or write of MongoDB.");
+
+    public static final ConfigOption<Integer> SCAN_FETCH_SIZE =
+            ConfigOptions.key("scan.fetch-size")
+                    .intType()
+                    .defaultValue(2048)
+                    .withDescription(
+                            "Gives the reader a hint as to the number of documents that should be fetched from the database per round-trip when reading. ");
+
+    public static final ConfigOption<Integer> SCAN_CURSOR_BATCH_SIZE =

Review Comment:
   - scan.fetch-size: The batch size fetched from `SplitReader`
   - [scan.cursor.batch-size](https://www.mongodb.com/docs/manual/reference/method/cursor.batchSize/#mongodb-method-cursor.batchSize): The batch size fetched from Mongo server.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011666412


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom

Review Comment:
   I mean how could a user pass custom options via the sinkConfiguration? You can't set arbitrary options in the MongoWriteOptions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017704601


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/config/MongoConnectionOptions.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.common.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.mongodb.ConnectionString;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The connection configuration class for MongoDB. */
+@PublicEvolving
+public class MongoConnectionOptions implements Serializable {
+
+    private final String uri;
+    private final String database;
+    private final String collection;
+
+    private MongoConnectionOptions(String uri, String database, String collection) {
+        this.uri = checkNotNull(uri);
+        this.database = checkNotNull(database);
+        this.collection = checkNotNull(collection);
+    }
+
+    public String getUri() {
+        return uri;
+    }
+
+    public String getDatabase() {
+        return database;
+    }
+
+    public String getCollection() {
+        return collection;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoConnectionOptions that = (MongoConnectionOptions) o;
+        return Objects.equals(uri, that.uri)
+                && Objects.equals(database, that.database)
+                && Objects.equals(collection, that.collection);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(uri, database, collection);
+    }
+
+    public static MongoConnectionOptionsBuilder builder() {
+        return new MongoConnectionOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoConnectionOptions}. */
+    @PublicEvolving
+    public static class MongoConnectionOptionsBuilder {
+        private String uri;
+        private String database;
+        private String collection;
+
+        /**
+         * Sets the connection string of MongoDB.
+         *
+         * @param uri connection string of MongoDB
+         * @return this builder
+         */
+        public MongoConnectionOptionsBuilder setUri(String uri) {
+            this.uri = new ConnectionString(uri).getConnectionString();
+            return this;

Review Comment:
   Yes, the constructor contains a series of validation logic, but does not resolve the host.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] rozza commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
rozza commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017963779


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.DefaultMongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible for writing records to a MongoDB collection.
+ *
+ * @param <IN> The type of the input elements.
+ */
+@Internal
+public class MongoWriter<IN> implements SinkWriter<IN> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriter.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+    private final MongoSinkContext sinkContext;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean flushOnCheckpoint;
+    private final List<WriteModel<BsonDocument>> bulkRequests = new ArrayList<>();
+    private final Collector<WriteModel<BsonDocument>> collector;
+    private final MongoClient mongoClient;
+
+    private boolean checkpointInProgress = false;
+    private volatile long lastSendTime = 0L;
+    private volatile long ackTime = Long.MAX_VALUE;
+
+    public MongoWriter(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            boolean flushOnCheckpoint,
+            Sink.InitContext initContext,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);
+        this.flushOnCheckpoint = flushOnCheckpoint;
+
+        checkNotNull(initContext);
+        this.mailboxExecutor = checkNotNull(initContext.getMailboxExecutor());
+
+        SinkWriterMetricGroup metricGroup = checkNotNull(initContext.metricGroup());
+        metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+
+        this.collector =
+                new CountingCollector<>(
+                        new ListCollector<>(this.bulkRequests),
+                        metricGroup.getNumRecordsSendCounter());
+
+        // Initialize the serialization schema.
+        this.sinkContext = new DefaultMongoSinkContext(initContext, writeOptions);
+        try {
+            SerializationSchema.InitializationContext initializationContext =
+                    initContext.asSerializationSchemaInitializationContext();
+            serializationSchema.open(initializationContext, sinkContext, writeOptions);
+        } catch (Exception e) {
+            throw new FlinkRuntimeException("Failed to open the MongoEmitter", e);
+        }
+
+        // Initialize the mongo client.
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    @Override
+    public void write(IN element, Context context) throws IOException, InterruptedException {
+        // do not allow new bulk writes until all actions are flushed
+        while (checkpointInProgress) {
+            mailboxExecutor.yield();
+        }
+        collector.collect(serializationSchema.serialize(element, sinkContext));
+        if (isOverMaxBatchSizeLimit() || isOverMaxBatchIntervalLimit()) {
+            doBulkWrite();
+        }
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException {
+        checkpointInProgress = true;
+        while (!bulkRequests.isEmpty() && (flushOnCheckpoint || endOfInput)) {
+            doBulkWrite();
+        }
+        checkpointInProgress = false;
+    }
+
+    @Override
+    public void close() {
+        mongoClient.close();
+    }
+
+    @VisibleForTesting
+    void doBulkWrite() throws IOException {
+        if (bulkRequests.isEmpty()) {
+            // no records to write
+            return;
+        }
+
+        int maxRetries = writeOptions.getMaxRetries();

Review Comment:
   Are all writes going to be idempotent? If not then rerunning multiple bulkwrite operations is likely to fail or produce multiple documents. This depends on the nature of the writes.
   
   The mongodb java driver supports: [retryable writes](https://www.mongodb.com/docs/manual/core/retryable-writes/) which has the ability to handle the retryability for transient networking errors.  In the MongoDB Kafka connector we removed the looping retry mechanism in favour of pushing users to set retryableWrites=true in the uri. See: [KAFKA-267](https://jira.mongodb.org/browse/KAFKA-267)



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitVectorSplitter.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoCommandException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.apache.commons.collections.CollectionUtils;
+import org.bson.BsonArray;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SPLIT_KEYS_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.UNAUTHORIZED_ERROR;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.splitVector;
+
+/**
+ * SplitVector Partitioner
+ *
+ * <p>Uses the SplitVector command to generate chunks for a collection. eg. <code>
+ * db.runCommand({splitVector:"inventory.products", keyPattern:{_id:1}, maxChunkSize:64})</code>
+ *
+ * <p>Requires splitVector privilege.
+ */
+@Internal
+public class MongoSplitVectorSplitter implements MongoSplitters.MongoSplitter {

Review Comment:
   Just an FYI - in the latest MongoDB Spark Connector we chose not add a SplitVector based partitioner due to the privilege requirements and because the API is marked as internal: See: [splitVector](https://www.mongodb.com/docs/manual/reference/command/splitVector/)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] twalthr commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
twalthr commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1043369287


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.annotation.Internal;
+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.MongoReadOptions;
+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;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
+import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** A {@link DynamicTableSource} for MongoDB. */
+@Internal
+public class MongoDynamicTableSource
+        implements ScanTableSource,
+                LookupTableSource,
+                SupportsProjectionPushDown,
+                SupportsLimitPushDown {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    @Nullable private final LookupCache lookupCache;
+    private final int lookupMaxRetries;
+    private final long lookupRetryIntervalMs;
+    private DataType physicalRowDataType;
+    private int limit = -1;
+
+    public MongoDynamicTableSource(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            @Nullable LookupCache lookupCache,
+            int lookupMaxRetries,
+            long lookupRetryIntervalMs,
+            DataType physicalRowDataType) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.lookupCache = lookupCache;
+        checkArgument(
+                lookupMaxRetries >= 0,
+                String.format(
+                        "The '%s' must be larger than or equal to 0.",
+                        LookupOptions.MAX_RETRIES.key()));
+        checkArgument(
+                lookupRetryIntervalMs > 0,
+                String.format("The '%s' must be larger than 0.", LOOKUP_RETRY_INTERVAL.key()));
+        this.lookupMaxRetries = lookupMaxRetries;
+        this.lookupRetryIntervalMs = lookupRetryIntervalMs;
+        this.physicalRowDataType = physicalRowDataType;
+    }
+
+    @Override
+    public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
+        final List<String> keyNames = new ArrayList<>(context.getKeys().length);
+        for (int[] innerKeyArr : context.getKeys()) {
+            Preconditions.checkArgument(
+                    innerKeyArr.length == 1, "MongoDB only support non-nested look up keys yet");
+            keyNames.add(DataType.getFieldNames(physicalRowDataType).get(innerKeyArr[0]));
+        }
+        final RowType rowType = (RowType) physicalRowDataType.getLogicalType();
+
+        MongoRowDataLookupFunction lookupFunction =
+                new MongoRowDataLookupFunction(
+                        connectionOptions,
+                        lookupMaxRetries,
+                        lookupRetryIntervalMs,
+                        DataType.getFieldNames(physicalRowDataType),
+                        DataType.getFieldDataTypes(physicalRowDataType),
+                        keyNames,
+                        rowType);
+        if (lookupCache != null) {
+            return PartialCachingLookupProvider.of(lookupFunction, lookupCache);
+        } else {
+            return LookupFunctionProvider.of(lookupFunction);
+        }
+    }
+
+    @Override
+    public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
+        final RowType rowType = (RowType) physicalRowDataType.getLogicalType();
+        final TypeInformation<RowData> typeInfo =
+                runtimeProviderContext.createTypeInformation(physicalRowDataType);
+
+        final MongoDeserializationSchema<RowData> deserializationSchema =
+                new MongoRowDataDeserializationSchema(rowType, typeInfo);
+
+        MongoSource<RowData> mongoSource =
+                MongoSource.<RowData>builder()
+                        .setUri(connectionOptions.getUri())
+                        .setDatabase(connectionOptions.getDatabase())
+                        .setCollection(connectionOptions.getCollection())
+                        .setFetchSize(readOptions.getFetchSize())
+                        .setCursorBatchSize(readOptions.getCursorBatchSize())
+                        .setNoCursorTimeout(readOptions.isNoCursorTimeout())
+                        .setPartitionStrategy(readOptions.getPartitionStrategy())
+                        .setPartitionSize(readOptions.getPartitionSize())
+                        .setSamplesPerPartition(readOptions.getSamplesPerPartition())
+                        .setLimit(limit)
+                        .setProjectedFields(DataType.getFieldNames(physicalRowDataType))
+                        .setDeserializationSchema(deserializationSchema)
+                        .build();
+
+        return SourceProvider.of(mongoSource);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode() {
+        return ChangelogMode.insertOnly();
+    }
+
+    @Override
+    public DynamicTableSource copy() {
+        return new MongoDynamicTableSource(
+                connectionOptions,
+                readOptions,
+                lookupCache,
+                lookupMaxRetries,
+                lookupRetryIntervalMs,
+                physicalRowDataType);
+    }
+
+    @Override
+    public String asSummaryString() {
+        return "MongoDB";
+    }
+
+    @Override
+    public void applyLimit(long limit) {
+        this.limit = (int) limit;
+    }
+
+    @Override
+    public boolean supportsNestedProjection() {
+        // planner doesn't support nested projection push down yet.
+        return false;
+    }
+
+    @Override
+    public void applyProjection(int[][] projectedFields, DataType producedDataType) {
+        this.physicalRowDataType = Projection.of(projectedFields).project(physicalRowDataType);

Review Comment:
   use `producedDataType` directly? 



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,677 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());

Review Comment:
   Creating a nullable converter regardless of the nullability of the logical type can be dangerous. It means that potentially nulls are travelling through the topology even though the schema says they don't. Either we do not allow `NOT NULL` constraints and check this in the `DynamicTableFactory` or we stick to the semantics and produce an error if an unexpected null occurs.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,677 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = new Object[in.size()];
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {

Review Comment:
   this logic here seems to be out of scope. BSON defines a boolean type and that's all we should support in this location. parsing ints or strings is the responsibility of the SQL engine and can be done by provided casting functions 



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSink.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.MongoSink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataSerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A {@link DynamicTableSink} for MongoDB. */
+@Internal
+public class MongoDynamicTableSink implements DynamicTableSink {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    @Nullable private final Integer parallelism;
+    private final DataType physicalRowDataType;
+    private final SerializableFunction<RowData, BsonValue> keyExtractor;
+
+    public MongoDynamicTableSink(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            @Nullable Integer parallelism,
+            DataType physicalRowDataType,
+            SerializableFunction<RowData, BsonValue> keyExtractor) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.parallelism = parallelism;
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.keyExtractor = checkNotNull(keyExtractor);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        return ChangelogMode.upsert();

Review Comment:
   Shouldn't the changelog mode depend on the primary key? If we are not in sync with `AppendOnlyKeyExtractor` an update-before and update-after might be partitioned as separate entities. When rereading they would appear as two independent records and thus source and sink do not consume/produce the same data.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java:
##########
@@ -0,0 +1,209 @@
+/*
+ * 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.annotation.Internal;
+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.MongoReadOptions;
+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;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
+import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** A {@link DynamicTableSource} for MongoDB. */
+@Internal
+public class MongoDynamicTableSource
+        implements ScanTableSource,
+                LookupTableSource,
+                SupportsProjectionPushDown,
+                SupportsLimitPushDown {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    @Nullable private final LookupCache lookupCache;
+    private final int lookupMaxRetries;
+    private final long lookupRetryIntervalMs;
+    private DataType physicalRowDataType;
+    private int limit = -1;
+
+    public MongoDynamicTableSource(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            @Nullable LookupCache lookupCache,
+            int lookupMaxRetries,
+            long lookupRetryIntervalMs,
+            DataType physicalRowDataType) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.lookupCache = lookupCache;
+        checkArgument(
+                lookupMaxRetries >= 0,
+                String.format(
+                        "The '%s' must be larger than or equal to 0.",
+                        LookupOptions.MAX_RETRIES.key()));
+        checkArgument(
+                lookupRetryIntervalMs > 0,
+                String.format("The '%s' must be larger than 0.", LOOKUP_RETRY_INTERVAL.key()));
+        this.lookupMaxRetries = lookupMaxRetries;
+        this.lookupRetryIntervalMs = lookupRetryIntervalMs;
+        this.physicalRowDataType = physicalRowDataType;
+    }
+
+    @Override
+    public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
+        final List<String> keyNames = new ArrayList<>(context.getKeys().length);
+        for (int[] innerKeyArr : context.getKeys()) {
+            Preconditions.checkArgument(
+                    innerKeyArr.length == 1, "MongoDB only support non-nested look up keys yet");
+            keyNames.add(DataType.getFieldNames(physicalRowDataType).get(innerKeyArr[0]));
+        }
+        final RowType rowType = (RowType) physicalRowDataType.getLogicalType();
+
+        MongoRowDataLookupFunction lookupFunction =
+                new MongoRowDataLookupFunction(
+                        connectionOptions,
+                        lookupMaxRetries,
+                        lookupRetryIntervalMs,
+                        DataType.getFieldNames(physicalRowDataType),
+                        DataType.getFieldDataTypes(physicalRowDataType),
+                        keyNames,
+                        rowType);
+        if (lookupCache != null) {
+            return PartialCachingLookupProvider.of(lookupFunction, lookupCache);
+        } else {
+            return LookupFunctionProvider.of(lookupFunction);
+        }
+    }
+
+    @Override
+    public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) {
+        final RowType rowType = (RowType) physicalRowDataType.getLogicalType();
+        final TypeInformation<RowData> typeInfo =
+                runtimeProviderContext.createTypeInformation(physicalRowDataType);
+
+        final MongoDeserializationSchema<RowData> deserializationSchema =
+                new MongoRowDataDeserializationSchema(rowType, typeInfo);
+
+        MongoSource<RowData> mongoSource =
+                MongoSource.<RowData>builder()
+                        .setUri(connectionOptions.getUri())
+                        .setDatabase(connectionOptions.getDatabase())
+                        .setCollection(connectionOptions.getCollection())
+                        .setFetchSize(readOptions.getFetchSize())
+                        .setCursorBatchSize(readOptions.getCursorBatchSize())
+                        .setNoCursorTimeout(readOptions.isNoCursorTimeout())
+                        .setPartitionStrategy(readOptions.getPartitionStrategy())
+                        .setPartitionSize(readOptions.getPartitionSize())
+                        .setSamplesPerPartition(readOptions.getSamplesPerPartition())
+                        .setLimit(limit)
+                        .setProjectedFields(DataType.getFieldNames(physicalRowDataType))
+                        .setDeserializationSchema(deserializationSchema)
+                        .build();
+
+        return SourceProvider.of(mongoSource);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode() {
+        return ChangelogMode.insertOnly();
+    }
+
+    @Override
+    public DynamicTableSource copy() {
+        return new MongoDynamicTableSource(
+                connectionOptions,
+                readOptions,
+                lookupCache,
+                lookupMaxRetries,
+                lookupRetryIntervalMs,
+                physicalRowDataType);
+    }
+
+    @Override
+    public String asSummaryString() {
+        return "MongoDB";
+    }
+
+    @Override
+    public void applyLimit(long limit) {
+        this.limit = (int) limit;
+    }
+
+    @Override
+    public boolean supportsNestedProjection() {
+        // planner doesn't support nested projection push down yet.
+        return false;
+    }
+
+    @Override
+    public void applyProjection(int[][] projectedFields, DataType producedDataType) {
+        this.physicalRowDataType = Projection.of(projectedFields).project(physicalRowDataType);

Review Comment:
   btw we try to have a naming convention for field names: `physicalRowDataType=table schema`, `producedDataType=fields effectively coming out of the table source`. let's rename the member variable in this class to `producedDataType`



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue() == 1;
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue() == 1L;
+        }
+        if (bsonValue.isString()) {
+            return Boolean.parseBoolean(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to boolean from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static byte convertToTinyInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (byte) 1 : (byte) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (byte) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (byte) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Byte.parseByte(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to tinyint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static short convertToSmallInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (short) 1 : (short) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (short) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (short) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Short.parseShort(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to smallint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static int convertToInt(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().intValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.intValue();
+            } else if (decimal128Value.isNegative()) {
+                return Integer.MIN_VALUE;
+            } else {
+                return Integer.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isDateTime()) {
+            return (int) Instant.ofEpochMilli(bsonValue.asDateTime().getValue()).getEpochSecond();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime();
+        }
+        if (bsonValue.isString()) {
+            return Integer.parseInt(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to integer from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static long convertToLong(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().longValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.longValue();
+            } else if (decimal128Value.isNegative()) {
+                return Long.MIN_VALUE;
+            } else {
+                return Long.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1L : 0L;
+        }
+        if (bsonValue.isDateTime()) {
+            return bsonValue.asDateTime().getValue();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime() * 1000L;
+        }
+        if (bsonValue.isString()) {
+            return Long.parseLong(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to long from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static double convertToDouble(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().doubleValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.doubleValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Double.MAX_VALUE;
+            } else {
+                return Double.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isString()) {
+            return Double.parseDouble(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to double from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static float convertToFloat(BsonValue bsonValue) {
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {

Review Comment:
   I share @zentol's opinion. I actually posted the same comment above. We should avoid conversion/casting logic in connectors but try to map types 1:1 where possible. I know that connectors have many types that Flink does not understand yet (or there is no way to express them), this is high on the agenda.
   
   Since MongoDB uses a string-based format, the easiest solution is that a user declares a field as STRING for data types that Flink does not support. This should make it possible to read values and post-process them with build-in JSON functions or UDFs. Is there a way in the BSON APIs to get the raw string-based out of it?



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSourceITCase.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.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 org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.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.BeforeEach;
+import org.junit.jupiter.api.Test;
+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.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+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.table.factories.FactoryUtil.CONNECTOR;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase for {@link MongoDynamicTableSource}. */
+@Testcontainers
+public class MongoDynamicTableSourceITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoDynamicTableSinkITCase.class);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    public static final String TEST_DATABASE = "test";
+    public static final String TEST_COLLECTION = "mongo_table_source";
+
+    private static MongoClient mongoClient;
+
+    public static StreamExecutionEnvironment env;
+    public static StreamTableEnvironment tEnv;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+
+        MongoCollection<BsonDocument> coll =
+                mongoClient
+                        .getDatabase(TEST_DATABASE)
+                        .getCollection(TEST_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        List<BsonDocument> testRecords = Arrays.asList(createTestData(1), createTestData(2));
+        coll.insertMany(testRecords);
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void before() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+    }
+
+    @Test
+    public void testSource() {
+        tEnv.executeSql(createTestDDl(null));
+
+        Iterator<Row> collected = tEnv.executeSql("SELECT * FROM mongo_source").collect();
+        List<String> result =
+                CollectionUtil.iteratorToList(collected).stream()
+                        .map(Row::toString)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        List<String> expected =
+                Stream.of(
+                                "+I[1, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]",
+                                "+I[2, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]")
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        assertThat(result).isEqualTo(expected);
+    }
+
+    @Test
+    public void testProject() {
+        tEnv.executeSql(createTestDDl(null));
+
+        Iterator<Row> collected = tEnv.executeSql("SELECT f1, f13 FROM mongo_source").collect();
+        List<String> result =
+                CollectionUtil.iteratorToList(collected).stream()
+                        .map(Row::toString)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        List<String> expected =
+                Stream.of("+I[2, +I[13]]", "+I[2, +I[13]]").sorted().collect(Collectors.toList());
+
+        assertThat(result).isEqualTo(expected);
+    }
+
+    @Test
+    public void testLimit() {
+        tEnv.executeSql(createTestDDl(null));
+
+        Iterator<Row> collected = tEnv.executeSql("SELECT * FROM mongo_source LIMIT 1").collect();
+        List<String> result =
+                CollectionUtil.iteratorToList(collected).stream()
+                        .map(Row::toString)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        Set<String> expected = new HashSet<>();
+        expected.add(
+                "+I[1, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]");
+        expected.add(
+                "+I[2, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]");
+
+        assertThat(result).hasSize(1);
+        assertThat(result).containsAnyElementsOf(expected);
+    }
+
+    @ParameterizedTest
+    @EnumSource(Caching.class)
+    public void testLookupJoin(Caching caching) throws Exception {
+        // Create MongoDB lookup table
+        Map<String, String> 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<Row> 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<Row> 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<String> result =
+                    CollectionUtil.iteratorToList(iterator).stream()
+                            .map(Row::toString)
+                            .sorted()
+                            .collect(Collectors.toList());
+            List<String> 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<String, LookupCacheManager.RefCountedCache> 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<RowData, Collection<RowData>> 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<String, String> extraOptions) {
+        Map<String, String> options = new HashMap<>();
+        options.put(CONNECTOR.key(), "mongodb");
+        options.put(URI.key(), MONGO_CONTAINER.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,",

Review Comment:
   Maybe I'm laking some MongoDB knowledge here but why do we allow `_id` to be BIGINT? Shouldn't this be always a hexadecimal string. I'm wondering whether we actually support round-trip semantics with this PR. A connector should feel like a database table which means the content of the sink can be re-read by the source ideally with exactly the same schema? Is this currently possible?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010737083


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/MongoSink.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+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.sink.writer.MongoWriter;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+
+import com.mongodb.client.model.WriteModel;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * MongoDB sink that requests multiple {@link WriteModel bulkRequests} against a cluster for each
+ * incoming element. The following example shows how to create a MongoSink receiving records of

Review Comment:
   How about change it to the following?
   ```
   Mongo sink converts each incoming element into MongoDB {@link WriteModel} (bulk write action) and
   bulk writes to mongodb when the number of actions is greater than bulkFlushMaxActions or the
   flush interval is greater than bulkFlushMaxActions.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011476914


##########
pom.xml:
##########
@@ -0,0 +1,457 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16-SNAPSHOT</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<hamcrest.version>1.3</hamcrest.version>
+		<assertj.version>3.21.0</assertj.version>
+		<archunit.version>0.22.0</archunit.version>
+		<testcontainers.version>1.17.2</testcontainers.version>
+		<mockito.version>3.4.6</mockito.version>
+
+		<japicmp.skip>false</japicmp.skip>
+		<japicmp.referenceVersion>1.15.0</japicmp.referenceVersion>
+
+		<slf4j.version>1.7.36</slf4j.version>
+		<log4j.version>2.17.2</log4j.version>
+
+		<flink.parent.artifactId>flink-connector-mongodb-parent</flink.parent.artifactId>
+	</properties>
+
+	<modules>
+		<module>flink-connector-mongodb</module>
+		<module>flink-sql-connector-mongodb</module>
+		<module>flink-connector-mongodb-e2e-tests</module>
+	</modules>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-force-shading</artifactId>
+			<version>${flink.shaded.version}</version>
+		</dependency>
+
+		<!-- Root dependencies for all projects -->
+
+		<!-- Logging API -->
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- 'javax.annotation' classes like '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.vintage</groupId>
+			<artifactId>junit-vintage-engine</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.assertj</groupId>
+			<artifactId>assertj-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.hamcrest</groupId>
+			<artifactId>hamcrest-all</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-inline</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-core</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Tests will have log4j as the default logging framework available -->
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-slf4j-impl</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- API bridge between log4j 1 and 2 -->
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-1.2-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-production</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.mongodb</groupId>
+				<artifactId>mongodb-driver-sync</artifactId>

Review Comment:
   In order to reduce dependencies, source and sink reuse the same driver.
   
   In most read scenarios, we need to wait for the clear result of the previous query before proceeding to the next step.
   Since we use the multi-reader model, there is no obvious performance improvement for asynchronous reading (except lookup), but there will be additional thread overhead.
   
   In the write scenario, asynchronous writing will indeed reduce the block time, but in order to prevent out-of-order writes, some additional synchronization control is required. 
   
   By the way, We can define a thread pool to implement asynchronous writing in future evolution, and it is not necessary to use the reactive driver.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#issuecomment-1298937872

   Thanks @zentol for the detailed review.
   I have made some changes to your comments.
   Other changes will be done today. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010733740


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoConstants.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonMaxKey;
+import org.bson.BsonMinKey;
+import org.bson.BsonValue;
+
+/** Constants for MongoDB. */
+@PublicEvolving

Review Comment:
   Changed to @Internal



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010751226


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom
+     *     options.
+     */
+    default void open(
+            SerializationSchema.InitializationContext initializationContext,
+            MongoSinkContext sinkContext,
+            MongoWriteOptions sinkConfiguration)

Review Comment:
   I am not sure whether users will want to obtain the parameters of config in the future evolution. In order to maintain the stability of the interface, the sinkConfiguration is provided. If this concern is unnecessary, I can delete it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#issuecomment-1299954327

   Hi @zentol.
   
   Thank you for taking the time to help with the review.
   Most of your comments have been addressed and some others would like to hear your suggestions.
   
   Please help review it again when you have time, thank you again.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017685301


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                retryIntervalMs,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private Integer parallelism;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkArgument(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkArgument(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0,
+                    "The sink max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the retry interval if writing records to database failed.
+         *
+         * @param retryIntervalMs the retry time interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setRetryInterval(long retryIntervalMs) {
+            checkArgument(
+                    retryIntervalMs > 0,
+                    "The retry interval (in milliseconds) must be larger than 0.");
+            this.retryIntervalMs = retryIntervalMs;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#AT_LEAST_ONCE}
+         *
+         * @param deliveryGuarantee which describes the record emission behaviour
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+            checkArgument(
+                    deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+                    "Mongo sink does not support the EXACTLY_ONCE guarantee.");
+            this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+            return this;
+        }
+
+        /**
+         * Sets the parallelism of the Mongo sink operator. By default, the parallelism is
+         * determined by the framework using the same parallelism of the upstream chained operator.
+         */
+        public MongoWriteOptionsBuilder setParallelism(int parallelism) {

Review Comment:
   @zentol 
   
   The parallelism parameter refers to jdbc and hbase connector.
   In fact, I am also a little confused about this.
   Should we delete the `parallelism` parameter ?
   
   ```java
   /** Options for the JDBC connector. */
   @Internal
   public class JdbcConnectorOptions extends JdbcConnectionOptions {
       private final String tableName;
       private final JdbcDialect dialect;
       private final @Nullable Integer parallelism;
   }
   
   public class JdbcDynamicTableSink {
       @Override
       public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
           final TypeInformation<RowData> rowDataTypeInformation =
                   context.createTypeInformation(physicalRowDataType);
           final JdbcOutputFormatBuilder builder = new JdbcOutputFormatBuilder();
   
           builder.setJdbcOptions(jdbcOptions);
           builder.setJdbcDmlOptions(dmlOptions);
           builder.setJdbcExecutionOptions(executionOptions);
           builder.setRowDataTypeInfo(rowDataTypeInformation);
           builder.setFieldDataTypes(
                   DataType.getFieldDataTypes(physicalRowDataType).toArray(new DataType[0]));
           return SinkFunctionProvider.of(
                   new GenericJdbcSinkFunction<>(builder.build()), jdbcOptions.getParallelism());
       }
   }
   ```
   
   ```java
   /** Options for HBase writing. */
   @Internal
   public class HBaseWriteOptions implements Serializable {
       private final long bufferFlushMaxSizeInBytes;
       private final long bufferFlushMaxRows;
       private final long bufferFlushIntervalMillis;
       private final Integer parallelism;
   }
   
   public class HBaseDynamicTableSink {
       @Override
       public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
           HBaseSinkFunction<RowData> sinkFunction =
                   new HBaseSinkFunction<>(
                           tableName,
                           hbaseConf,
                           new RowDataToMutationConverter(hbaseTableSchema, nullStringLiteral),
                           writeOptions.getBufferFlushMaxSizeInBytes(),
                           writeOptions.getBufferFlushMaxRows(),
                           writeOptions.getBufferFlushIntervalMillis());
           return SinkFunctionProvider.of(sinkFunction, writeOptions.getParallelism());
       }
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017713074


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                retryIntervalMs,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private Integer parallelism;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkArgument(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkArgument(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0,
+                    "The sink max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the retry interval if writing records to database failed.
+         *
+         * @param retryIntervalMs the retry time interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setRetryInterval(long retryIntervalMs) {
+            checkArgument(
+                    retryIntervalMs > 0,
+                    "The retry interval (in milliseconds) must be larger than 0.");
+            this.retryIntervalMs = retryIntervalMs;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#AT_LEAST_ONCE}
+         *
+         * @param deliveryGuarantee which describes the record emission behaviour
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+            checkArgument(
+                    deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+                    "Mongo sink does not support the EXACTLY_ONCE guarantee.");
+            this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+            return this;
+        }
+
+        /**
+         * Sets the parallelism of the Mongo sink operator. By default, the parallelism is
+         * determined by the framework using the same parallelism of the upstream chained operator.
+         */
+        public MongoWriteOptionsBuilder setParallelism(int parallelism) {

Review Comment:
   So it seems that `FactoryUtil#SINK_PARALLELISM` is a common constant that factories can use, and the factories just pass this parallelism to the `*TableSink` which then _again_ forwards it to `SinkFunctionProvider`, from which the Table API finally derives the parallelism.
   
   Seems to be a bit of a mess in my eyes.
   I'd suggest to not put it into the `WriteOptions` (because that's shared across APIs) and keep it focused on things that the _actual_ sink needs.
   We can pass the parallelism explicitly to the TableSink constructor instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025492765


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable
+    public static BsonDocument readCollectionMetadata(
+            MongoClient mongoClient, MongoNamespace namespace) {
+        MongoCollection<BsonDocument> collection =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(COLLECTIONS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        return collection
+                .find(eq(ID_FIELD, namespace.getFullName()))
+                .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD))
+                .first();
+    }
+
+    public static boolean isValidShardedCollection(BsonDocument collectionMetadata) {
+        return collectionMetadata != null
+                && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue();
+    }

Review Comment:
   The `collectionMetadata` was retrived from `config.collections`.
   The [config.collections](https://www.mongodb.com/docs/v5.0/reference/config-database/#mongodb-data-config.collections) stores a document for each sharded (only sharded) collection in the cluster.
   
   As commented above, we need to filter out sharded collections that have been dropped.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025581498


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/PartitionStrategy.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+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;
+
+/** MongoSplitStrategy that can be chosen. */
+@PublicEvolving
+public enum PartitionStrategy implements DescribedEnum {

Review Comment:
   pros:
   - sample: Fast, No extra permission required.
   - split-vector: Fast; Chunk results are even.
   - sharded: Very fast; No calculation is required, directly read the [chunks](https://www.mongodb.com/docs/v5.0/reference/config-database/#mongodb-data-config.chunks) that have been split.
   
   cons:
   - sample: Chunk results may be uneven.
   - split-vector: `splitVector` permission is required.
   - sharded: Only for sharded collection; The`read` permission  for `config` database is required; The chunk size depends on MongoDB server cannot be adjusted by this connector.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032963748


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoShardedSplitter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.mongodb.MongoException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isShardedCollectionDropped;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readChunks;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readCollectionMetadata;
+
+/**
+ * Sharded Partitioner
+ *
+ * <p>Uses the chunks collection and partitions the collection based on the sharded collections
+ * chunk ranges.
+ *
+ * <p>The following config collections' read privilege is required.
+ *
+ * <ul>
+ *   <li>config.collections
+ *   <li>config.chunks
+ * </ul>
+ */
+@Internal
+public class MongoShardedSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+    public static final MongoShardedSplitter INSTANCE = new MongoShardedSplitter();
+
+    private MongoShardedSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoClient mongoClient = splitContext.getMongoClient();
+
+        List<BsonDocument> chunks;
+        Optional<BsonDocument> collectionMetadata;
+        try {
+            collectionMetadata = readCollectionMetadata(mongoClient, namespace);
+            if (!collectionMetadata.isPresent()) {
+                LOG.error(
+                        "Do sharded split failed, collection {} does not appear to be sharded.",
+                        namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, %s is not a sharded collection.",
+                                namespace));
+            }
+
+            if (isShardedCollectionDropped(collectionMetadata.get())) {
+                LOG.error("Do sharded split failed, collection {} was dropped.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format("Do sharded split failed, %s was dropped.", namespace));
+            }
+
+            chunks = readChunks(mongoClient, collectionMetadata.get());
+            if (chunks.isEmpty()) {
+                LOG.error("Do sharded split failed, chunks of {} is empty.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, chunks of %s is empty.", namespace));
+            }
+        } catch (MongoException e) {
+            LOG.error(
+                    "Read chunks from {} failed with error message: {}", namespace, e.getMessage());
+            throw new FlinkRuntimeException(e);
+        }
+
+        List<MongoScanSourceSplit> sourceSplits = new ArrayList<>(chunks.size());
+        for (int i = 0; i < chunks.size(); i++) {
+            BsonDocument chunk = chunks.get(i);

Review Comment:
   Yes, there will be some extra fields. Do we need to use them to encode the split name of `MongoScanSourceSplit`?
   
   - `_id` field represents the primary key recorded in [config.chunks](https://www.mongodb.com/docs/upcoming/reference/config-database/#mongodb-data-config.chunks)
   - `uuid` field represents the record of [config.collections](https://www.mongodb.com/docs/upcoming/reference/config-database/#mongodb-data-config.collections)
   - [shard](https://www.mongodb.com/docs/manual/sharding/#sharded-cluster) field identifies the shard in the cluster that "owns" the chunk.
   
   ```javascript
   [{
       "_id" : ObjectId("63838f8368a7363e32c5287b"), 
       "uuid" : UUID("986f129e-9c6a-4ecb-b407-dbde92e6e37e"),
       "min" : {
           "user_id" : "user_830",
           "product_no" : NumberLong(830)
       },
       "max" : {
           "user_id" : { "$maxKey" : 1 },
           "product_no" : { "$maxKey" : 1 }
       },
       "shard" : "rs0-shard",
       "lastmod" : Timestamp(1, 3),
       "history" : [ 
           {
               "validAfter" : Timestamp(1669566339, 13),
               "shard" : "rs0-shard"
           }
       ]
   },
   {
       "_id" : ObjectId("63838f8368a7363e32c52899"),
       "uuid" : UUID("986f129e-9c6a-4ecb-b407-dbde92e6e37e"),
       "min" : {
           "user_id" : "user_2986",
           "product_no" : NumberLong(2986)
       },
       "max" : {
           "user_id" : "user_6560",
           "product_no" : NumberLong(6560)
       },
       "shard" : "rs0-shard",
       "lastmod" : Timestamp(1, 5),
       "history" : [ 
           {
               "validAfter" : Timestamp(1669566339, 13),
               "shard" : "rs0-shard"
           }
       ]
   }]
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032983150


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoScanSourceSplitReader.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.api.connector.source.SourceReaderContext;
+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.MongoReadOptions;
+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 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;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.project;
+
+/** An split reader implements {@link SplitReader} for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSourceSplitReader implements MongoSourceSplitReader<MongoSourceSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSourceSplitReader.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    private final SourceReaderContext readerContext;
+    @Nullable private final List<String> projectedFields;
+    private final int limit;
+
+    private boolean closed = false;
+    private boolean finished = false;
+    private MongoClient mongoClient;
+    private MongoCursor<BsonDocument> currentCursor;
+    private MongoScanSourceSplit currentSplit;
+
+    public MongoScanSourceSplitReader(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            @Nullable List<String> projectedFields,
+            int limit,
+            SourceReaderContext context) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.projectedFields = projectedFields;
+        this.limit = limit;
+        this.readerContext = context;
+    }
+
+    @Override
+    public RecordsWithSplitIds<BsonDocument> fetch() throws IOException {
+        if (closed) {
+            throw new IllegalStateException("Cannot fetch records from a closed split reader");
+        }
+
+        RecordsBySplits.Builder<BsonDocument> builder = new RecordsBySplits.Builder<>();
+
+        // Return when no split registered to this reader.
+        if (currentSplit == null) {
+            return builder.build();
+        }
+
+        currentCursor = getOrCreateCursor();
+        int fetchSize = readOptions.getFetchSize();
+
+        try {
+            for (int recordNum = 0; recordNum < fetchSize; recordNum++) {
+                if (currentCursor.hasNext()) {
+                    builder.add(currentSplit, currentCursor.next());
+                } else {
+                    builder.addFinishedSplit(currentSplit.splitId());
+                    finished = true;
+                    break;
+                }
+            }
+            return builder.build();
+        } catch (MongoException e) {
+            throw new IOException("Scan records form MongoDB failed", e);
+        } finally {
+            if (finished) {
+                currentSplit = null;
+                releaseCursor();
+            }
+        }
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<MongoSourceSplit> 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()));
+        }
+
+        MongoSourceSplit sourceSplit = splitsChanges.splits().get(0);
+        if (!(sourceSplit instanceof MongoScanSourceSplit)) {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "The SourceSplit type of %s is not supported.",
+                            sourceSplit.getClass()));
+        }
+
+        this.currentSplit = (MongoScanSourceSplit) sourceSplit;
+        this.finished = false;
+    }
+
+    @Override
+    public void wakeUp() {}

Review Comment:
   We took the approach of closing the cursor  to cancel blocked hasNext(), next() when the wakeUp method is invoked.
   
   `MongoCursor#close`
   > Despite this interface being non-thread-safe, close() is allowed to be called concurrently with any method of the cursor, including itself. This is useful to cancel blocked hasNext(), next(). This method is idempotent.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032991900


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.table.converter.BsonToRowDataConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.LookupFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+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;
+import org.bson.conversions.Bson;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+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.project;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A lookup function for {@link MongoDynamicTableSource}. */
+@Internal
+public class MongoRowDataLookupFunction extends LookupFunction {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoRowDataLookupFunction.class);
+    private static final long serialVersionUID = 1L;
+
+    private final MongoConnectionOptions connectionOptions;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+
+    private final List<String> fieldNames;
+    private final List<String> keyNames;
+
+    private final BsonToRowDataConverters.BsonToRowDataConverter mongoRowConverter;
+    private final RowDataToBsonConverters.RowDataToBsonConverter lookupKeyRowConverter;
+
+    private transient MongoClient mongoClient;
+
+    public MongoRowDataLookupFunction(
+            MongoConnectionOptions connectionOptions,
+            int maxRetries,
+            long retryIntervalMs,
+            List<String> fieldNames,
+            List<DataType> fieldTypes,
+            List<String> keyNames,
+            RowType rowType) {
+        checkNotNull(fieldNames, "No fieldNames supplied.");
+        checkNotNull(fieldTypes, "No fieldTypes supplied.");
+        checkNotNull(keyNames, "No keyNames supplied.");
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.fieldNames = fieldNames;
+        this.mongoRowConverter = BsonToRowDataConverters.createNullableConverter(rowType);
+
+        this.keyNames = keyNames;
+        LogicalType[] keyTypes =
+                this.keyNames.stream()
+                        .map(
+                                s -> {
+                                    checkArgument(
+                                            fieldNames.contains(s),

Review Comment:
   Thanks. Table API guarantees this to be the case. I'll remove that check.
   > org.apache.flink.table.api.ValidationException: SQL validation failed. From line 1, column 131 to line 1, column 133: Column 'f18' not found in table 'D'
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1094408110


##########
flink-connector-mongodb-e2e-tests/src/test/resources/e2e_append_only.sql:
##########
@@ -0,0 +1,44 @@
+--/*
+-- * Licensed to the Apache Software Foundation (ASF) under one
+-- * or more contributor license agreements.  See the NOTICE file
+-- * distributed with this work for additional information
+-- * regarding copyright ownership.  The ASF licenses this file
+-- * to you under the Apache License, Version 2.0 (the
+-- * "License"); you may not use this file except in compliance
+-- * with the License.  You may obtain a copy of the License at
+-- *
+-- *     http://www.apache.org/licenses/LICENSE-2.0
+-- *
+-- * Unless required by applicable law or agreed to in writing, software
+-- * distributed under the License is distributed on an "AS IS" BASIS,
+-- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- * See the License for the specific language governing permissions and
+-- * limitations under the License.
+-- */
+
+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',

Review Comment:
   Can we use a random port for mongodb? This would reduce the chances of this test failing locally when by chance something else is currently running on that port.



##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,163 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Use fat jar so we don't need to create a user-jar. -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-sql-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<profiles>
+		<profile>
+			<id>run-end-to-end-tests</id>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<executions>
+							<execution>
+								<id>end-to-end-tests</id>
+								<phase>integration-test</phase>
+								<goals>
+									<goal>test</goal>
+								</goals>
+								<configuration>
+									<includes>
+										<include>**/*.*</include>
+									</includes>
+									<systemPropertyVariables>
+										<moduleDir>${project.basedir}</moduleDir>
+									</systemPropertyVariables>
+								</configuration>
+							</execution>
+						</executions>
+					</plugin>
+				</plugins>
+			</build>
+		</profile>
+	</profiles>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-deploy-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>default-test</id>
+						<phase>none</phase>
+					</execution>
+					<execution>
+						<id>integration-tests</id>
+						<phase>none</phase>
+					</execution>
+				</executions>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-dependency-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>copy</id>
+						<phase>pre-integration-test</phase>
+						<goals>
+							<goal>copy</goal>
+						</goals>
+						<configuration>
+							<artifactItems>
+								<artifactItem>
+									<groupId>org.apache.flink</groupId>
+									<artifactId>flink-sql-connector-mongodb</artifactId>
+									<version>${project.version}</version>
+									<destFileName>sql-mongodb.jar</destFileName>
+									<type>jar</type>
+									<outputDirectory>${project.build.directory}/dependencies</outputDirectory>
+								</artifactItem>
+							</artifactItems>
+						</configuration>
+					</execution>
+					<execution>
+						<id>store-classpath-in-target-for-tests</id>
+						<phase>package</phase>
+						<goals>
+							<goal>build-classpath</goal>
+						</goals>
+						<configuration>
+							<outputFile>${project.build.directory}/hadoop.classpath</outputFile>
+							<excludeGroupIds>org.apache.flink</excludeGroupIds>
+						</configuration>
+					</execution>

Review Comment:
   This appears unused.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.table.MongoKeyExtractor;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import org.bson.BsonType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Set;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoValidationUtils.class);
+
+    public static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES =
+            EnumSet.of(
+                    LogicalTypeRoot.CHAR,
+                    LogicalTypeRoot.VARCHAR,
+                    LogicalTypeRoot.BOOLEAN,
+                    LogicalTypeRoot.DECIMAL,
+                    LogicalTypeRoot.INTEGER,
+                    LogicalTypeRoot.BIGINT,
+                    LogicalTypeRoot.DOUBLE,
+                    LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE,
+                    LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE,
+                    LogicalTypeRoot.INTERVAL_YEAR_MONTH,
+                    LogicalTypeRoot.INTERVAL_DAY_TIME);
+
+    private static final Set<LogicalTypeRoot> DENIED_PRIMARY_KEY_TYPES =
+            EnumSet.of(
+                    LogicalTypeRoot.BINARY,
+                    LogicalTypeRoot.VARBINARY,
+                    LogicalTypeRoot.TINYINT,
+                    LogicalTypeRoot.SMALLINT,
+                    LogicalTypeRoot.FLOAT,
+                    LogicalTypeRoot.DATE,
+                    LogicalTypeRoot.TIME_WITHOUT_TIME_ZONE,
+                    LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE,
+                    LogicalTypeRoot.ARRAY,
+                    LogicalTypeRoot.MULTISET,
+                    LogicalTypeRoot.MAP,
+                    LogicalTypeRoot.ROW,
+                    LogicalTypeRoot.DISTINCT_TYPE,
+                    LogicalTypeRoot.STRUCTURED_TYPE,
+                    LogicalTypeRoot.NULL,
+                    LogicalTypeRoot.RAW,
+                    LogicalTypeRoot.SYMBOL,
+                    LogicalTypeRoot.UNRESOLVED);
+
+    /**
+     * Checks that the table does not have a primary key defined on illegal types. In MongoDB the
+     * primary key is used to calculate the MongoDB document _id, which may be of any {@link
+     * BsonType} other than a {@link BsonType#ARRAY}. Its value must be unique and immutable in the
+     * collection.
+     *
+     * <p>MongoDB creates a unique index on the _id field during the creation of a collection. There
+     * are also some constraints on the primary key index. For more detailed introduction, you can
+     * refer to <a
+     * href="https://www.mongodb.com/docs/manual/reference/limits/#mongodb-limit-Index-Key-Limit">
+     * Index Key Limit</a>.
+     *
+     * <ul>
+     *   <li>Before MongoDB 4.2, the total size of an index entry, which can include structural
+     *       overhead depending on the BSON type, must be less than 1024 bytes.
+     *   <li>Starting in version 4.2, MongoDB removes the Index Key Limit.
+     * </ul>
+     *
+     * <p>As of now it is extracted by {@link MongoKeyExtractor} according to the primary key
+     * specified by the Flink table schema.
+     *
+     * <ul>
+     *   <li>When there's only a single field in the specified primary key, we convert the field
+     *       data to bson value as _id of the corresponding document.
+     *   <li>When there's multiple fields in the specified primary key, we convert and composite
+     *       these fields into a {@link BsonType#DOCUMENT} as the _id of the corresponding document.
+     *       For example, if have a primary key statement <code>PRIMARY KEY (f1, f2) NOT ENFORCED
+     *       </code>, the extracted _id will be the form like <code>_id: {f1: v1, f2: v2}</code>
+     * </ul>
+     *
+     * <p>The illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link
+     * LogicalTypeRoot#RAW} type and other types that cannot be converted to {@link BsonType} by
+     * {@link RowDataToBsonConverters}.
+     */
+    public static void validatePrimaryKey(DataType primaryKeyDataType) {
+        List<DataType> fieldDataTypes = DataType.getFieldDataTypes(primaryKeyDataType);
+        List<DataType> illegalTypes = new ArrayList<>();
+        for (DataType fieldType : fieldDataTypes) {
+            LogicalTypeRoot typeRoot = fieldType.getLogicalType().getTypeRoot();
+            if (!ALLOWED_PRIMARY_KEY_TYPES.contains(typeRoot)) {
+                illegalTypes.add(fieldType);
+                if (!DENIED_PRIMARY_KEY_TYPES.contains(typeRoot)) {
+                    LOG.warn(
+                            "Detected newly added root type {} that should to be explicitly accepted or rejected.",

Review Comment:
   Maybe include something like "Please reach out to the Flink maintainers."



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.enumerator.splitter.MongoSplitters;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSplitAssigner implements MongoSplitAssigner {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSplitAssigner.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+
+    private final LinkedList<String> remainingCollections;
+    private final List<String> alreadyProcessedCollections;
+    private final List<MongoScanSourceSplit> remainingScanSplits;
+    private final Map<String, MongoScanSourceSplit> assignedScanSplits;
+    private boolean initialized;
+
+    private transient MongoSplitters mongoSplitters;
+
+    public MongoScanSplitAssigner(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            MongoSourceEnumState sourceEnumState) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.remainingCollections = new LinkedList<>(sourceEnumState.getRemainingCollections());

Review Comment:
   It seems odd that this uses a LinkedList to somewhat achieve queue semantics while remainingSplits is some opaque list.
   As a whole I'm wondering if these really should be lists or not a `deque`.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonDocument;
+import org.bson.BsonType;
+import org.bson.BsonValue;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.DEFAULT_JSON_WRITER_SETTINGS;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    public static BsonToRowDataConverter createConverter(LogicalType type) {

Review Comment:
   This should only accept `RowType`.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonDocument;
+import org.bson.BsonType;
+import org.bson.BsonValue;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.DEFAULT_JSON_WRITER_SETTINGS;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {

Review Comment:
   This interface is problematic. It exists for 2 distinct use-cases but mixes the 2 and pays for it with loss of type safety.
   
   There is 1 use-case outside of this class where want to convert a `BsonValue` to a `RowData`, as the name implies. Currently this always requires an unchecked cast.
   
   But it's also used internally to extract arbitrary objects from a BsonValue, which could just as well be a `SerializableFunction<BsonValue, Object>`.
   
   It's just not really a `RowData` convert, but an arbitrary mapping between Table-supported types and BsonValues.
   



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/RowDataToBsonConverters.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonNull;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.json.JsonParseException;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+
+/** Tool class used to convert from {@link RowData} to {@link BsonValue}. */
+@Internal
+public class RowDataToBsonConverters {
+
+    // --------------------------------------------------------------------------------
+    // Runtime Converters
+    // --------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts objects of Flink Table & SQL internal data structures to
+     * corresponding {@link BsonValue} data structures.
+     */
+    @FunctionalInterface
+    public interface RowDataToBsonConverter extends Serializable {
+        BsonValue convert(Object value);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    public static RowDataToBsonConverter createConverter(LogicalType type) {
+        return wrapIntoNullSafeInternalConverter(createInternalConverter(type), type);
+    }
+
+    private static RowDataToBsonConverter wrapIntoNullSafeInternalConverter(
+            RowDataToBsonConverter rowDataToBsonConverter, LogicalType type) {
+        return new RowDataToBsonConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public BsonValue convert(Object value) {
+                if (value == null || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) {
+                    if (type.isNullable()) {
+                        return BsonNull.VALUE;
+                    } else {
+                        throw new IllegalArgumentException(
+                                "The column type is <"
+                                        + type
+                                        + ">, but a null value is being written into it");
+                    }
+                } else {
+                    return rowDataToBsonConverter.convert(value);
+                }
+            }
+        };
+    }
+
+    private static RowDataToBsonConverter createInternalConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new RowDataToBsonConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public BsonValue convert(Object value) {
+                        return null;

Review Comment:
   Why doesn't this return BsonNull.VALUE?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/RowDataToBsonConverters.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonNull;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.json.JsonParseException;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+
+/** Tool class used to convert from {@link RowData} to {@link BsonValue}. */
+@Internal
+public class RowDataToBsonConverters {
+
+    // --------------------------------------------------------------------------------
+    // Runtime Converters
+    // --------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts objects of Flink Table & SQL internal data structures to
+     * corresponding {@link BsonValue} data structures.
+     */
+    @FunctionalInterface
+    public interface RowDataToBsonConverter extends Serializable {
+        BsonValue convert(Object value);
+    }

Review Comment:
   Similar concerns as for BsonToRowDataConverters.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriterITCase.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.connector.mongodb.testutils.MongoTestUtil;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.DeleteOneModel;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.UpdateOneModel;
+import com.mongodb.client.model.UpdateOptions;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+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.util.Optional;
+import java.util.OptionalLong;
+
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreNotWritten;
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link MongoWriter}. */
+@Testcontainers
+public class MongoWriterITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriterITCase.class);
+
+    private static final String TEST_DATABASE = "test_writer";
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    private static MongoClient mongoClient;
+    private static MetricListener metricListener;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void setUp() {
+        metricListener = new MetricListener();
+    }
+
+    @Test
+    void testWriteOnBulkFlush() throws Exception {
+        final String collection = "test-bulk-flush-without-checkpoint";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 5;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+
+            // Ignore flush on checkpoint
+            writer.flush(false);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3, 4);
+
+            // Trigger flush
+            writer.write(buildMessage(5), null);
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5);
+
+            writer.write(buildMessage(6), null);
+            assertThatIdsAreNotWritten(collectionOf(collection), 6);
+
+            // Force flush
+            writer.doBulkWrite();
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5, 6);
+        }
+    }
+
+    @Test
+    void testWriteOnBatchIntervalFlush() throws Exception {
+        final String collection = "test-bulk-flush-with-interval";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = -1;
+        final int batchIntervalMs = 1000;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+            writer.doBulkWrite();
+        }
+
+        assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4);
+    }
+
+    @Test
+    void testWriteOnCheckpoint() throws Exception {
+        final String collection = "test-bulk-flush-with-checkpoint";
+        final boolean flushOnCheckpoint = true;
+        final int batchSize = -1;
+        final int batchIntervalMs = -1;
+
+        // Enable flush on checkpoint
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3);
+
+            // Trigger flush
+            writer.flush(false);
+
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3);
+        }
+    }
+
+    @Test
+    void testIncrementRecordsSendMetric() throws Exception {
+        final String collection = "test-inc-records-send";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Counter> recordsSend =
+                    metricListener.getCounter(MetricNames.NUM_RECORDS_SEND);
+            writer.write(buildMessage(1), null);
+            // Update existing index
+            writer.write(buildMessage(2, "u"), null);
+            // Delete index
+            writer.write(buildMessage(3, "d"), null);
+
+            writer.doBulkWrite();
+
+            assertThat(recordsSend.isPresent()).isTrue();
+            assertThat(recordsSend.get().getCount()).isEqualTo(3L);
+        }
+    }
+
+    @Test
+    void testCurrentSendTime() throws Exception {
+        final String collection = "test-current-send-time";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Gauge<Long>> currentSendTime =
+                    metricListener.getGauge("currentSendTime");
+
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+
+            writer.doBulkWrite();
+
+            assertThat(currentSendTime.isPresent()).isTrue();
+            assertThat(currentSendTime.get().getValue()).isGreaterThan(0L);

Review Comment:
   This is basically guaranteed to fail at some point because we're dealing with such small time measurements.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSinkITCase.java:
##########
@@ -0,0 +1,510 @@
+/*
+ * 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.EnvironmentSettings;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.expressions.Expression;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.types.Row;
+import org.apache.flink.types.RowKind;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.Filters;
+import org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDbPointer;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonJavaScript;
+import org.bson.BsonJavaScriptWithScope;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonString;
+import org.bson.BsonSymbol;
+import org.bson.BsonTimestamp;
+import org.bson.Document;
+import org.bson.types.Binary;
+import org.bson.types.Decimal128;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+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.sql.Timestamp;
+import java.time.Instant;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.regex.Pattern;
+
+import static org.apache.flink.table.api.Expressions.row;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** IT tests for {@link MongoDynamicTableSink}. */
+@Testcontainers
+public class MongoDynamicTableSinkITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoDynamicTableSinkITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .build());
+
+    private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterEach
+    public void tearDown() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @Test
+    public void testSinkWithAllSupportedTypes() throws ExecutionException, InterruptedException {
+        String database = "test";
+        String collection = "sink_with_all_supported_types";
+
+        TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
+
+        tEnv.executeSql(
+                String.join(
+                        "\n",
+                        Arrays.asList(
+                                "CREATE TABLE mongo_sink",
+                                "(",
+                                "  _id BIGINT,",
+                                "  f1 STRING,",
+                                "  f2 BOOLEAN,",
+                                "  f3 BINARY,",
+                                "  f4 INTEGER,",
+                                "  f5 TIMESTAMP_LTZ(6),",
+                                "  f6 TIMESTAMP(3),",
+                                "  f7 DOUBLE,",
+                                "  f8 DECIMAL(10, 2),",
+                                "  f9 MAP<STRING, INTEGER>,",
+                                "  f10 ROW<k INTEGER>,",
+                                "  f11 ARRAY<STRING>,",
+                                "  f12 ARRAY<ROW<k STRING>>,",
+                                "  PRIMARY KEY (_id) NOT ENFORCED",
+                                ") WITH (",
+                                getConnectorSql(database, collection),
+                                ")")));
+
+        Instant now = Instant.now();
+        tEnv.fromValues(
+                        DataTypes.ROW(
+                                DataTypes.FIELD("_id", DataTypes.BIGINT()),
+                                DataTypes.FIELD("f1", DataTypes.STRING()),
+                                DataTypes.FIELD("f2", DataTypes.BOOLEAN()),
+                                DataTypes.FIELD("f3", DataTypes.BINARY(1)),
+                                DataTypes.FIELD("f4", DataTypes.INT()),
+                                DataTypes.FIELD("f5", DataTypes.TIMESTAMP_LTZ(6)),
+                                DataTypes.FIELD("f6", DataTypes.TIMESTAMP(3)),
+                                DataTypes.FIELD("f7", DataTypes.DOUBLE()),
+                                DataTypes.FIELD("f8", DataTypes.DECIMAL(10, 2)),
+                                DataTypes.FIELD(
+                                        "f9", DataTypes.MAP(DataTypes.STRING(), DataTypes.INT())),
+                                DataTypes.FIELD(
+                                        "f10",
+                                        DataTypes.ROW(DataTypes.FIELD("k", DataTypes.INT()))),
+                                DataTypes.FIELD("f11", DataTypes.ARRAY(DataTypes.STRING())),
+                                DataTypes.FIELD(
+                                        "f12",
+                                        DataTypes.ARRAY(
+                                                DataTypes.ROW(
+                                                        DataTypes.FIELD(
+                                                                "K", DataTypes.STRING()))))),
+                        Row.of(
+                                1L,
+                                "ABCDE",
+                                true,
+                                new byte[] {(byte) 3},
+                                6,
+                                now,
+                                Timestamp.from(now),
+                                10.10d,
+                                new BigDecimal("11.11"),
+                                Collections.singletonMap("k", 12),
+                                Row.of(13),
+                                Arrays.asList("14_1", "14_2"),
+                                Arrays.asList(Row.of("15_1"), Row.of("15_2"))))
+                .executeInsert("mongo_sink")
+                .await();
+
+        MongoCollection<Document> coll =
+                mongoClient.getDatabase(database).getCollection(collection);
+
+        Document actual = coll.find(Filters.eq("_id", 1L)).first();
+
+        Document expected =
+                new Document("_id", 1L)
+                        .append("f1", "ABCDE")
+                        .append("f2", true)
+                        .append("f3", new Binary(new byte[] {(byte) 3}))
+                        .append("f4", 6)
+                        .append("f5", Date.from(now))
+                        .append("f6", Date.from(now))
+                        .append("f7", 10.10d)
+                        .append("f8", new Decimal128(new BigDecimal("11.11")))
+                        .append("f9", new Document("k", 12))
+                        .append("f10", new Document("k", 13))
+                        .append("f11", Arrays.asList("14_1", "14_2"))
+                        .append(
+                                "f12",
+                                Arrays.asList(
+                                        new Document("k", "15_1"), new Document("k", "15_2")));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testRoundTripReadAndSink() throws ExecutionException, InterruptedException {
+        String database = "test";
+        String sourceCollection = "test_round_trip_source";
+        String sinkCollection = "test_round_trip_sink";
+
+        BsonDocument testData =
+                new BsonDocument("f1", new BsonString("ABCDE"))
+                        .append("f2", new BsonBoolean(true))
+                        .append("f3", new BsonBinary(new byte[] {(byte) 3}))
+                        .append("f4", new BsonInt32(32))
+                        .append("f5", new BsonInt64(64L))
+                        .append("f6", new BsonDouble(128.128d))
+                        .append("f7", new BsonDecimal128(new Decimal128(new BigDecimal("256.256"))))
+                        .append("f8", new BsonDateTime(Instant.now().toEpochMilli()))
+                        .append("f9", new BsonTimestamp((int) Instant.now().getEpochSecond(), 100))
+                        .append(
+                                "f10",
+                                new BsonRegularExpression(Pattern.compile("^9$").pattern(), "i"))
+                        .append("f11", new BsonJavaScript("function() { return 10; }"))
+                        .append(
+                                "f12",
+                                new BsonJavaScriptWithScope(
+                                        "function() { return 11; }", new BsonDocument()))
+                        .append("f13", new BsonDbPointer("test.test", new ObjectId()))
+                        .append("f14", new BsonSymbol("symbol"))
+                        .append(
+                                "f15",
+                                new BsonArray(Arrays.asList(new BsonInt32(1), new BsonInt32(2))))
+                        .append("f16", new BsonDocument("k", new BsonInt32(32)));
+
+        MongoCollection<BsonDocument> sourceColl =
+                mongoClient
+                        .getDatabase(database)
+                        .getCollection(sourceCollection)
+                        .withDocumentClass(BsonDocument.class);
+        sourceColl.insertOne(testData);
+
+        TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
+
+        tEnv.executeSql(
+                String.format(
+                        "CREATE TABLE mongo_source (\n"
+                                + "`_id` STRING,\n"
+                                + "`f1` STRING,\n"
+                                + "`f2` BOOLEAN,\n"
+                                + "`f3` BINARY,\n"
+                                + "`f4` INTEGER,\n"
+                                + "`f5` BIGINT,\n"
+                                + "`f6` DOUBLE,\n"
+                                + "`f7` DECIMAL(10, 3),\n"
+                                + "`f8` TIMESTAMP_LTZ(3),\n"
+                                + "`f9` STRING,\n"
+                                + "`f10` STRING,\n"
+                                + "`f11` STRING,\n"
+                                + "`f12` STRING,\n"
+                                + "`f13` STRING,\n"
+                                + "`f14` STRING,\n"
+                                + "`f15` ARRAY<INTEGER>,\n"
+                                + "`f16` ROW<k INTEGER>,\n"
+                                + " PRIMARY KEY (_id) NOT ENFORCED\n"
+                                + ") WITH ( %s )",
+                        getConnectorSql(database, sourceCollection)));
+
+        tEnv.executeSql(
+                String.format(
+                        "CREATE TABLE mongo_sink WITH ( %s ) LIKE mongo_source",
+                        getConnectorSql(database, sinkCollection)));
+
+        tEnv.executeSql("insert into mongo_sink select * from mongo_source").await();
+
+        MongoCollection<BsonDocument> sinkColl =
+                mongoClient
+                        .getDatabase(database)
+                        .getCollection(sinkCollection)
+                        .withDocumentClass(BsonDocument.class);
+
+        BsonDocument actual = sinkColl.find().first();
+
+        assertThat(actual).isEqualTo(testData);
+    }
+
+    @Test
+    public void testSinkWithAllRowKind() throws ExecutionException, InterruptedException {
+        String database = "test";
+        String collection = "test_sink_with_all_row_kind";
+
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+        DataStream<Row> sourceStream =
+                env.fromCollection(
+                                Arrays.asList(
+                                        Row.ofKind(RowKind.INSERT, 1L, "Alice"),
+                                        Row.ofKind(RowKind.DELETE, 1L, "Alice"),
+                                        Row.ofKind(RowKind.INSERT, 2L, "Bob"),
+                                        Row.ofKind(RowKind.UPDATE_BEFORE, 2L, "Bob"),
+                                        Row.ofKind(RowKind.UPDATE_AFTER, 2L, "Tom")))
+                        .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())
+                        .build();
+
+        Table sourceTable = tEnv.fromChangelogStream(sourceStream, sourceSchema);
+        tEnv.createTemporaryView("value_source", sourceTable);
+
+        tEnv.executeSql(
+                String.format(
+                        "CREATE TABLE mongo_sink (\n"
+                                + "`_id` BIGINT,\n"
+                                + "`name` STRING,\n"
+                                + " PRIMARY KEY (_id) NOT ENFORCED\n"
+                                + ") WITH ( %s )",
+                        getConnectorSql(database, collection)));
+
+        tEnv.executeSql("insert into mongo_sink select * from value_source").await();
+
+        MongoCollection<Document> coll =
+                mongoClient.getDatabase(database).getCollection(collection);
+
+        List<Document> expected =
+                Collections.singletonList(new Document("_id", 2L).append("name", "Tom"));
+
+        List<Document> actual = coll.find().into(new ArrayList<>());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testSinkWithReservedId() throws Exception {
+        String database = "test";
+        String collection = "sink_with_reserved_id";
+
+        TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
+
+        tEnv.executeSql(
+                String.format(
+                        "CREATE TABLE mongo_sink ("
+                                + "_id STRING NOT NULL,\n"
+                                + "f1 STRING NOT NULL,\n"
+                                + "PRIMARY KEY (_id) NOT ENFORCED\n"
+                                + ")\n"
+                                + "WITH (%s)",
+                        getConnectorSql(database, collection)));
+
+        ObjectId objectId = new ObjectId();
+        tEnv.fromValues(row(objectId.toHexString(), "r1"), row("str", "r2"))
+                .executeInsert("mongo_sink")
+                .await();
+
+        MongoCollection<Document> coll =
+                mongoClient.getDatabase(database).getCollection(collection);
+
+        List<Document> actual = new ArrayList<>();
+        coll.find(Filters.in("_id", objectId, "str")).into(actual);
+
+        Document[] expected =
+                new Document[] {
+                    new Document("_id", objectId).append("f1", "r1"),
+                    new Document("_id", "str").append("f1", "r2")
+                };
+        assertThat(actual).containsExactlyInAnyOrder(expected);
+    }
+
+    @Test
+    public void testSinkWithoutPrimaryKey() throws Exception {
+        String database = "test";
+        String collection = "sink_without_primary_key";
+
+        TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
+
+        tEnv.executeSql(
+                String.format(
+                        "CREATE TABLE mongo_sink (" + "f1 STRING NOT NULL\n" + ")\n" + "WITH (%s)",
+                        getConnectorSql(database, collection)));
+
+        tEnv.fromValues(row("d1"), row("d1")).executeInsert("mongo_sink").await();
+
+        MongoCollection<Document> coll =
+                mongoClient.getDatabase(database).getCollection(collection);
+
+        List<Document> actual = new ArrayList<>();
+        coll.find().into(actual);
+
+        assertThat(actual).hasSize(2);
+        for (Document doc : actual) {
+            assertThat(doc.get("f1")).isEqualTo("d1");
+        }
+    }
+
+    @Test
+    public void testSinkWithNonCompositePrimaryKey() throws Exception {
+        String database = "test";
+        String collection = "sink_with_non_composite_pk";
+
+        Instant now = Instant.now();
+        List<Expression> testValues =
+                Collections.singletonList(
+                        row(1L, true, "ABCDE", 12.12d, 4, Timestamp.from(now), now));

Review Comment:
   ```suggestion
                           row(2L, true, "ABCDE", 12.12d, 4, Timestamp.from(now), now));
   ```
   Let's use some value that doesnt happen to coincide with the generated(?) `_id`.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.enumerator.splitter.MongoSplitters;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSplitAssigner implements MongoSplitAssigner {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSplitAssigner.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+
+    private final LinkedList<String> remainingCollections;
+    private final List<String> alreadyProcessedCollections;
+    private final List<MongoScanSourceSplit> remainingScanSplits;
+    private final Map<String, MongoScanSourceSplit> assignedScanSplits;
+    private boolean initialized;
+
+    private transient MongoSplitters mongoSplitters;
+
+    public MongoScanSplitAssigner(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            MongoSourceEnumState sourceEnumState) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.remainingCollections = new LinkedList<>(sourceEnumState.getRemainingCollections());
+        this.alreadyProcessedCollections = sourceEnumState.getAlreadyProcessedCollections();
+        this.remainingScanSplits = sourceEnumState.getRemainingScanSplits();
+        this.assignedScanSplits = sourceEnumState.getAssignedScanSplits();
+        this.initialized = sourceEnumState.isInitialized();
+    }
+
+    @Override
+    public void open() {
+        LOG.info("Mongo scan split assigner is opening.");
+        if (!initialized) {
+            String collectionId =
+                    String.format(
+                            "%s.%s",
+                            connectionOptions.getDatabase(), connectionOptions.getCollection());
+            remainingCollections.add(collectionId);
+            mongoSplitters = new MongoSplitters(connectionOptions, readOptions);
+            initialized = true;
+        }
+    }
+
+    @Override
+    public Optional<MongoSourceSplit> getNext() {
+        if (!remainingScanSplits.isEmpty()) {
+            // return remaining splits firstly
+            Iterator<MongoScanSourceSplit> iterator = remainingScanSplits.iterator();
+            MongoScanSourceSplit split = iterator.next();
+            iterator.remove();
+            assignedScanSplits.put(split.splitId(), split);
+            return Optional.of(split);
+        } else {
+            // it's turn for next collection
+            String nextCollection = remainingCollections.pollFirst();
+            if (nextCollection != null) {
+                // split the given collection into chunks (scan splits)
+                Collection<MongoScanSourceSplit> splits =
+                        mongoSplitters.split(new MongoNamespace(nextCollection));
+                remainingScanSplits.addAll(splits);
+                alreadyProcessedCollections.add(nextCollection);
+                return getNext();
+            } else {
+                return Optional.empty();
+            }
+        }
+    }
+
+    @Override
+    public void addSplitsBack(Collection<MongoSourceSplit> splits) {
+        for (MongoSourceSplit split : splits) {
+            if (split instanceof MongoScanSourceSplit) {
+                remainingScanSplits.add((MongoScanSourceSplit) split);
+                // we should remove the add-backed splits from the assigned list,
+                // because they are failed
+                assignedScanSplits.remove(split.splitId());
+            }
+        }
+    }
+
+    @Override
+    public MongoSourceEnumState snapshotState(long checkpointId) {
+        return new MongoSourceEnumState(
+                remainingCollections,
+                alreadyProcessedCollections,
+                remainingScanSplits,
+                assignedScanSplits,
+                initialized);
+    }
+
+    @Override
+    public boolean noMoreSplits() {
+        return initialized && remainingCollections.isEmpty() && remainingScanSplits.isEmpty();
+    }

Review Comment:
   I'd rather add a Precondition that initialized is true. We don't want this to be called in any other situation, and as-is we might mask such a problem.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.source.enumerator.MongoSourceEnumState;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoSourceSplit}. */
+@Internal
+public interface MongoSplitAssigner extends Serializable {
+
+    /**
+     * Called to open the assigner to acquire any resources, like threads or network connections.
+     */
+    void open();
+
+    /**
+     * Called to close the assigner, in case it holds on to any resources, like threads or network
+     * connections.
+     */
+    void close() throws IOException;
+
+    /** Gets the next split. */
+    Optional<MongoSourceSplit> getNext();

Review Comment:
   Document that. Specifically it's relationship to `noMoreSplits()`.
   



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/serialization/MongoRowDataDeserializationSchema.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table.serialization;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+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.bson.BsonDocument;
+
+/** Deserializer that {@link BsonDocument} to flink internal {@link RowData}. */

Review Comment:
   ```suggestion
   /** Deserializer that maps {@link BsonDocument} to {@link RowData}. */
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.mongodb.MongoException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Closeable {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
+
+    private final MongoReadOptions readOptions;
+    private final MongoClient mongoClient;
+
+    public MongoSplitters(MongoConnectionOptions connectionOptions, MongoReadOptions readOptions) {
+        this.readOptions = readOptions;
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    public Collection<MongoScanSourceSplit> split(MongoNamespace namespace) {
+        BsonDocument collStats;
+        try {
+            collStats = MongoUtils.collStats(mongoClient, namespace);
+        } catch (MongoException e) {
+            LOG.error("Execute collStats command failed, with error message: {}", e.getMessage());
+            throw new FlinkRuntimeException(e);
+        }
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(readOptions, mongoClient, namespace, collStats);
+
+        switch (readOptions.getPartitionStrategy()) {
+            case SINGLE:
+                return MongoSingleSplitter.INSTANCE.split(splitContext);

Review Comment:
   These could all be static methods; the singleton pattern doesn't really gives as any benefit.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/testutils/MongoShardedContainers.java:
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.flink.connector.mongodb.testutils;
+
+import com.github.dockerjava.api.command.InspectContainerResponse;
+import org.junit.jupiter.api.extension.AfterAllCallback;
+import org.junit.jupiter.api.extension.BeforeAllCallback;
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.MongoDBContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.io.IOException;
+
+/** Sharded Containers. */
+public class MongoShardedContainers implements BeforeAllCallback, AfterAllCallback {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoShardedContainers.class);
+
+    private static final int MONGODB_INTERNAL_PORT = 27017;
+
+    private static final String CONFIG_REPLICA_SET_NAME = "rs-config-0";
+    private static final String SHARD_REPLICA_SET_NAME = "rs-shard-0";
+
+    private static final String CONFIG_HOSTNAME = "config-0";
+    private static final String SHARD_HOSTNAME = "shard-0";
+    private static final String ROUTER_HOSTNAME = "router-0";
+
+    private final MongoDBContainer configSrv;
+    private final MongoDBContainer shardSrv;
+    private final MongoDBContainer router;
+
+    MongoShardedContainers(DockerImageName dockerImageName, Network network) {
+        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))
+                        .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))
+                        .withNetwork(network)
+                        .withNetworkAliases(SHARD_HOSTNAME)
+                        .withLogConsumer(logConsumer);
+        this.router =
+                new MongoRouterContainer(dockerImageName)
+                        .withCreateContainerCmdModifier(it -> it.withHostName(ROUTER_HOSTNAME))
+                        .dependsOn(configSrv, shardSrv)
+                        .withNetwork(network)
+                        .withNetworkAliases(ROUTER_HOSTNAME)
+                        .withLogConsumer(logConsumer);
+    }
+
+    public void start() {
+        LOG.info("Starting ConfigSrv container");
+        configSrv.start();
+        LOG.info("Starting ShardSrv container");
+        shardSrv.start();
+        LOG.info("Starting Router containers");
+        router.start();
+    }
+
+    public void close() {
+        router.stop();
+        shardSrv.stop();
+        configSrv.stop();
+    }

Review Comment:
   Why are these public? can we just inline them into the callback methods?



##########
flink-connector-mongodb/src/test/resources/log4j2-test.properties:
##########
@@ -0,0 +1,28 @@
+################################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+# limitations under the License.
+################################################################################
+
+# Set root logger level to OFF to not flood build logs
+# set manually to INFO for debugging purposes
+rootLogger.level = INFO

Review Comment:
   ```suggestion
   rootLogger.level = OFF
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/RowDataToBsonConverters.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonNull;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.json.JsonParseException;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+
+/** Tool class used to convert from {@link RowData} to {@link BsonValue}. */
+@Internal
+public class RowDataToBsonConverters {
+
+    // --------------------------------------------------------------------------------
+    // Runtime Converters
+    // --------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts objects of Flink Table & SQL internal data structures to
+     * corresponding {@link BsonValue} data structures.
+     */
+    @FunctionalInterface
+    public interface RowDataToBsonConverter extends Serializable {
+        BsonValue convert(Object value);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    public static RowDataToBsonConverter createConverter(LogicalType type) {

Review Comment:
   Should only accept RowData



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.time.Duration;
+
+/**
+ * Base options for the MongoDB connector. Needs to be public so that the {@link
+ * org.apache.flink.table.api.TableDescriptor} can access it.
+ */
+@PublicEvolving
+public class MongoConnectorOptions {
+
+    private MongoConnectorOptions() {}
+
+    public static final ConfigOption<String> URI =
+            ConfigOptions.key("uri")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the connection uri of MongoDB.");
+
+    public static final ConfigOption<String> DATABASE =
+            ConfigOptions.key("database")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the database to read or write of MongoDB.");
+
+    public static final ConfigOption<String> COLLECTION =
+            ConfigOptions.key("collection")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the collection to read or write of MongoDB.");
+
+    public static final ConfigOption<Integer> SCAN_FETCH_SIZE =
+            ConfigOptions.key("scan.fetch-size")
+                    .intType()
+                    .defaultValue(2048)
+                    .withDescription(
+                            "Gives the reader a hint as to the number of documents that should be fetched from the database per round-trip when reading. ");
+
+    public static final ConfigOption<Integer> SCAN_CURSOR_BATCH_SIZE =

Review Comment:
   This isn't enough information for users, and current descriptions are highly misleading.
   
   scan size states "the number of documents that should be fetched from the database", but it's rather the number of records _consumed_ in one fetch() call from one or more retrieved batches.
   But this is a highly technical description that users won't know what to do with.
   
   
   We just have to expand a bit on the trade-offs.
   batch size reduces the # of calls to MongoDB at the cost of higher memory consumption in Flink and higher risk of throwing away fetched data (when fetch-size is significantly lower and a failure occurs), while fetch size is all about keeping the source responsive to checkpointing.
   
   I do wonder though whether fetch-size should be necessary. To me it really points to a problem on how `fetch()` is implemented / designed.
   What would be the overhead of just existing fetch() every 10 records?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.RowData.FieldGetter;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = ID_FIELD;
+
+    private static final AppendOnlyKeyExtractor APPEND_ONLY_KEY_EXTRACTOR =
+            new AppendOnlyKeyExtractor();
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverter primaryKeyConverter;
+
+    private final FieldGetter primaryKeyGetter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createConverter(primaryKeyType);
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            this.primaryKeyGetter =
+                    rowData -> ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+        } else {
+            this.primaryKeyGetter = RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+        }
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        Object rowKeyValue = primaryKeyGetter.getFieldOrNull(rowData);
+        checkNotNull(rowKeyValue, "Primary key value is null of RowData: " + rowData);
+        BsonValue keyValue = primaryKeyConverter.convert(rowKeyValue);
+        if (!isCompoundPrimaryKey(primaryKeyIndexes) && keyValue.isString()) {
+            String keyString = keyValue.asString().getValue();
+            // Try to restore MongoDB's ObjectId from string.
+            if (ObjectId.isValid(keyString)) {
+                keyValue = new BsonObjectId(new ObjectId(keyString));
+            }
+        }
+        return keyValue;
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reservedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // Primary key is not declared and reserved _id is not present.
+        if (!primaryKey.isPresent() && !reservedId.isPresent()) {
+            return APPEND_ONLY_KEY_EXTRACTOR;
+        }
+
+        if (reservedId.isPresent()) {
+            // Ambiguous keys being used due to the presence of an _id field.
+            if (!primaryKey.isPresent()
+                    || isCompoundPrimaryKey(primaryKeyIndexes)
+                    || !primaryKeyContainsReservedId(primaryKey.get())) {
+                throw new IllegalArgumentException(
+                        "Ambiguous keys being used due to the presence of an _id field.");

Review Comment:
   Add instructions for how to resolve that.
   
   For example, "Either use the _id column as the key, or rename the _id column".



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.RowData.FieldGetter;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = ID_FIELD;
+
+    private static final AppendOnlyKeyExtractor APPEND_ONLY_KEY_EXTRACTOR =
+            new AppendOnlyKeyExtractor();
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverter primaryKeyConverter;
+
+    private final FieldGetter primaryKeyGetter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createConverter(primaryKeyType);
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            this.primaryKeyGetter =
+                    rowData -> ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+        } else {
+            this.primaryKeyGetter = RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+        }
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        Object rowKeyValue = primaryKeyGetter.getFieldOrNull(rowData);
+        checkNotNull(rowKeyValue, "Primary key value is null of RowData: " + rowData);
+        BsonValue keyValue = primaryKeyConverter.convert(rowKeyValue);
+        if (!isCompoundPrimaryKey(primaryKeyIndexes) && keyValue.isString()) {
+            String keyString = keyValue.asString().getValue();
+            // Try to restore MongoDB's ObjectId from string.
+            if (ObjectId.isValid(keyString)) {
+                keyValue = new BsonObjectId(new ObjectId(keyString));
+            }
+        }
+        return keyValue;
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reservedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // Primary key is not declared and reserved _id is not present.
+        if (!primaryKey.isPresent() && !reservedId.isPresent()) {
+            return APPEND_ONLY_KEY_EXTRACTOR;
+        }
+
+        if (reservedId.isPresent()) {
+            // Ambiguous keys being used due to the presence of an _id field.
+            if (!primaryKey.isPresent()
+                    || isCompoundPrimaryKey(primaryKeyIndexes)
+                    || !primaryKeyContainsReservedId(primaryKey.get())) {
+                throw new IllegalArgumentException(
+                        "Ambiguous keys being used due to the presence of an _id field.");
+            }
+        }
+
+        DataType primaryKeyType;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            DataType physicalRowDataType = resolvedSchema.toPhysicalRowDataType();
+            primaryKeyType = Projection.of(primaryKeyIndexes).project(physicalRowDataType);
+        } else {
+            int primaryKeyIndex = primaryKeyIndexes[0];
+            Optional<Column> column = resolvedSchema.getColumn(primaryKeyIndex);
+            if (!column.isPresent()) {
+                throw new IllegalStateException(
+                        String.format(
+                                "No primary key column found with index '%s'.", primaryKeyIndex));
+            }
+            primaryKeyType = column.get().getDataType();
+        }
+
+        MongoValidationUtils.validatePrimaryKey(primaryKeyType);
+
+        return new MongoKeyExtractor(primaryKeyType.getLogicalType(), primaryKeyIndexes);
+    }
+
+    private static boolean isCompoundPrimaryKey(int[] primaryKeyIndexes) {
+        return primaryKeyIndexes.length > 1;
+    }
+
+    private static boolean primaryKeyContainsReservedId(UniqueConstraint primaryKey) {
+        return primaryKey.getColumns().contains(RESERVED_ID);
+    }
+
+    /**
+     * It behaves as append-only when no primary key is declared and reserved _id is not present. We
+     * use static class instead of lambda for a reason here. It is necessary because the maven shade

Review Comment:
   ```suggestion
        * use static class instead of lambda because the maven shade
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.RowData.FieldGetter;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {

Review Comment:
   missing SerialVersion UID



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+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.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+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.COLLECTION;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.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.CONNECTOR;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test for {@link MongoDynamicTableSource} and {@link MongoDynamicTableSink} created by {@link
+ * MongoDynamicTableFactory}.
+ */
+public class MongoDynamicTableFactoryTest {
+
+    private static final ResolvedSchema SCHEMA =
+            new ResolvedSchema(
+                    Arrays.asList(
+                            Column.physical("aaa", DataTypes.INT().notNull()),
+                            Column.physical("bbb", DataTypes.STRING().notNull()),
+                            Column.physical("ccc", DataTypes.DOUBLE()),
+                            Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
+                            Column.physical("eee", DataTypes.TIMESTAMP(3))),
+                    Collections.emptyList(),
+                    UniqueConstraint.primaryKey("name", Arrays.asList("bbb", "aaa")));
+
+    @Test
+    public void testMongoCommonProperties() {
+        Map<String, String> properties = getRequiredOptions();
+
+        // validation for source
+        DynamicTableSource actualSource = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expectedSource =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+        assertThat(actualSource).isEqualTo(expectedSource);
+
+        // validation for sink

Review Comment:
   split the test case instead of adding such comments.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.

Review Comment:
   Why aren't we just doing this?
   
   ```
   // the number of elements we want each split to read
   numDocumentsPerPartition = partitionSizeInBytes / avgObjSizeInBytes
   sampleCount = count / numDocumentsPerPartition
   totalSplitCount = sampleCount + 1
   ```
   
   What are multiple sample per split supposed to give us? Is the MongoDB sampling not good enough that we need a higher resolution?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoJsonDeserializationSchema.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.reader.deserializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import org.bson.BsonDocument;
+import org.bson.json.JsonMode;
+
+import java.util.Optional;
+
+/**
+ * A schema bridge for deserializing the MongoDB's {@code BsonDocument} to MongoDB's {@link
+ * JsonMode}'s RELAXED Json string.
+ */
+@PublicEvolving
+public class MongoJsonDeserializationSchema implements MongoDeserializationSchema<String> {

Review Comment:
   > Used to convert BsonDocument to Json format
   
   That's not a use-case, but what it does.
   Who do we expect to use it? Is it only for testing? Production usage?



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java:
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+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.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+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.COLLECTION;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.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.CONNECTOR;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test for {@link MongoDynamicTableSource} and {@link MongoDynamicTableSink} created by {@link
+ * MongoDynamicTableFactory}.
+ */
+public class MongoDynamicTableFactoryTest {
+
+    private static final ResolvedSchema SCHEMA =
+            new ResolvedSchema(
+                    Arrays.asList(
+                            Column.physical("aaa", DataTypes.INT().notNull()),
+                            Column.physical("bbb", DataTypes.STRING().notNull()),
+                            Column.physical("ccc", DataTypes.DOUBLE()),
+                            Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
+                            Column.physical("eee", DataTypes.TIMESTAMP(3))),
+                    Collections.emptyList(),
+                    UniqueConstraint.primaryKey("name", Arrays.asList("bbb", "aaa")));
+
+    @Test
+    public void testMongoCommonProperties() {
+        Map<String, String> properties = getRequiredOptions();
+
+        // validation for source
+        DynamicTableSource actualSource = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expectedSource =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+        assertThat(actualSource).isEqualTo(expectedSource);
+
+        // validation for sink
+        DynamicTableSink actualSink = createTableSink(SCHEMA, properties);
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+        MongoDynamicTableSink expectedSink =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.getPrimaryKey().isPresent(),
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+        assertThat(actualSink).isEqualTo(expectedSink);
+    }
+
+    @Test
+    public void testMongoReadProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put(SCAN_FETCH_SIZE.key(), "1024");
+        properties.put(SCAN_CURSOR_BATCH_SIZE.key(), "2048");
+        properties.put(SCAN_CURSOR_NO_TIMEOUT.key(), "false");
+        properties.put(SCAN_PARTITION_STRATEGY.key(), "split-vector");
+        properties.put(SCAN_PARTITION_SIZE.key(), "128m");
+        properties.put(SCAN_PARTITION_SAMPLES.key(), "5");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoReadOptions readOptions =
+                MongoReadOptions.builder()
+                        .setFetchSize(1024)
+                        .setCursorBatchSize(2048)
+                        .setNoCursorTimeout(false)
+                        .setPartitionStrategy(PartitionStrategy.SPLIT_VECTOR)
+                        .setPartitionSize(MemorySize.ofMebiBytes(128))
+                        .setSamplesPerPartition(5)
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        readOptions,
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoLookupProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put(LookupOptions.CACHE_TYPE.key(), "PARTIAL");
+        properties.put(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE.key(), "10s");
+        properties.put(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS.key(), "20s");
+        properties.put(LookupOptions.PARTIAL_CACHE_CACHE_MISSING_KEY.key(), "false");
+        properties.put(LookupOptions.PARTIAL_CACHE_MAX_ROWS.key(), "15213");
+        properties.put(LookupOptions.MAX_RETRIES.key(), "10");
+        properties.put(LOOKUP_RETRY_INTERVAL.key(), "20ms");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        DefaultLookupCache.fromConfig(Configuration.fromMap(properties)),
+                        10,
+                        20,
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put(BUFFER_FLUSH_MAX_ROWS.key(), "1001");
+        properties.put(BUFFER_FLUSH_INTERVAL.key(), "2min");
+        properties.put(DELIVERY_GUARANTEE.key(), "at-least-once");
+        properties.put(SINK_MAX_RETRIES.key(), "5");
+        properties.put(SINK_RETRY_INTERVAL.key(), "2s");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoWriteOptions writeOptions =
+                MongoWriteOptions.builder()
+                        .setBatchSize(1001)
+                        .setBatchIntervalMs(TimeUnit.MINUTES.toMillis(2))
+                        .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+                        .setMaxRetries(5)
+                        .setRetryIntervalMs(TimeUnit.SECONDS.toMillis(2))
+                        .build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.getPrimaryKey().isPresent(),
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkWithParallelism() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.parallelism", "2");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        2,
+                        SCHEMA.getPrimaryKey().isPresent(),
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoValidation() {
+        // fetch size lower than 1
+        Map<String, String> properties = getRequiredOptions();
+        properties.put(SCAN_FETCH_SIZE.key(), "0");
+
+        Map<String, String> finalProperties1 = properties;

Review Comment:
   You could make this test a lot nicer with a utility method like `getRequiredOptionsWithSetting(String, String)`.
   ```
   assertThatThrownBy(() -> createTableSource(SCHEMA, getRequiredOptionsWithSetting(SCAN_FETCH_SIZE.key(), "0"))
           .hasStackTraceContaining("The fetch size must be larger than 0.");
   ```
   
   Bonus points for adding another `assertValidationRejects(String, String,String)` method.
   
   ```
   assertValidationRejects(SCAN_FETCH_SIZE.key(), "0", "The fetch size must be larger than 0.");
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/MongoSinkITCase.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+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.test.junit5.MiniClusterExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 javax.annotation.Nullable;
+
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** IT cases for {@link MongoSink}. */
+@Testcontainers
+public class MongoSinkITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSinkITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    private static final String TEST_DATABASE = "test_sink";
+
+    private static boolean failed;
+
+    private static MongoClient mongoClient;
+
+    @BeforeAll
+    static void setUp() {
+        failed = false;
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void tearDown() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @ParameterizedTest
+    @EnumSource(DeliveryGuarantee.class)
+    void testWriteToMongoWithDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee)
+            throws Exception {
+        final String index = "test-sink-with-delivery-" + deliveryGuarantee;
+        boolean failure = false;
+        try {
+            runTest(index, false, deliveryGuarantee, null);
+        } catch (IllegalArgumentException e) {
+            failure = true;
+            assertThat(deliveryGuarantee).isSameAs(DeliveryGuarantee.EXACTLY_ONCE);

Review Comment:
   I still think we shouldn't even test exactly_once, apart from it being rejected (in a dedicated test).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1096955404


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/RowDataToBsonConverters.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonNull;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.json.JsonParseException;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+
+/** Tool class used to convert from {@link RowData} to {@link BsonValue}. */
+@Internal
+public class RowDataToBsonConverters {
+
+    // --------------------------------------------------------------------------------
+    // Runtime Converters
+    // --------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts objects of Flink Table & SQL internal data structures to
+     * corresponding {@link BsonValue} data structures.
+     */
+    @FunctionalInterface
+    public interface RowDataToBsonConverter extends Serializable {
+        BsonValue convert(Object value);
+    }

Review Comment:
   Redefined `RowDataToBsonConverter` as below.
   
   ```java
       @FunctionalInterface
       public interface RowDataToBsonConverter extends Serializable {
           BsonDocument convert(RowData rowData);
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] boring-cyborg[bot] commented on pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "boring-cyborg[bot] (via GitHub)" <gi...@apache.org>.
boring-cyborg[bot] commented on PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#issuecomment-1422854589

   Awesome work, congrats on your first merged pull request!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1030405674


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable
+    public static BsonDocument readCollectionMetadata(
+            MongoClient mongoClient, MongoNamespace namespace) {
+        MongoCollection<BsonDocument> collection =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(COLLECTIONS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        return collection
+                .find(eq(ID_FIELD, namespace.getFullName()))
+                .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD))
+                .first();
+    }
+
+    public static boolean isValidShardedCollection(BsonDocument collectionMetadata) {
+        return collectionMetadata != null
+                && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue();
+    }
+
+    public static List<BsonDocument> readChunks(
+            MongoClient mongoClient, BsonDocument collectionMetadata) {
+        MongoCollection<BsonDocument> chunks =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(CHUNKS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        Bson filter =
+                or(
+                        new BsonDocument(NAMESPACE_FIELD, collectionMetadata.get(ID_FIELD)),
+                        // MongoDB 4.9.0 removed ns field of config.chunks collection, using

Review Comment:
   My question was less aimed at what the driver supports, but rather with MongoDB server versions are actively supported by the project.
   For example, if 3.x is no longer supported by MongoDB (irrespective of whether the driver supports it), then we should consider also not supporting 3.x.
   The immediate benefit would be that (I think) we'd no longer have to bother with the `dropped` flag at all. According to https://www.mongodb.com/support-policy/lifecycles 3.6 already reached EOL in April 20**21**.



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -134,13 +134,13 @@ private List<Document> readAllBackupOrders(MongoDatabase db, int expect) throws
         return backupOrders;
     }
 
-    private List<Document> mockOrders(int ordersCount) {

Review Comment:
   I'm not sure why you changed this; here's no need for it. There is are `ElementsOf` variants of the assertj matchers that accept an iterable.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reversedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // It behaves as append-only when no primary key is declared and no reversed _id is present.
+        // We use anonymous classes instead of lambdas for a reason here. It is
+        // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas
+        // (MSHADE-260).
+        if (!primaryKey.isPresent() && !reversedId.isPresent()) {
+            return new SerializableFunction<RowData, BsonValue>() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public BsonValue apply(RowData rowData) {
+                    return null;
+                }
+            };
+        }
+
+        if (reversedId.isPresent()) {
+            // Primary key should be declared as (_id) when the mongo reversed _id is present.
+            if (!primaryKey.isPresent()
+                    || isCompoundPrimaryKey(primaryKeyIndexes)
+                    || !primaryKeyContainsReversedId(primaryKey.get())) {
+                throw new IllegalArgumentException(
+                        "The primary key should be declared as (_id) when mongo reversed _id field is present");

Review Comment:
   That's an interesting question. We don't seem to handle this case in a special way in the ES connector; we use the `PRIMARY KEY` as the id and insert the document as is with it's _id_ field; the final behavior will depend on Elasticsearch.
   
   I don't really like that behavior because as you said it is ambiguous; we should fail early if the schema contains an _id_ field and it's not the sole primary key.
   
   My comment was rather about the error message. You are explicitly suggesting `(_id)` as an alternative in this case; but is that really the best option? Maybe they just made a mistake w.r.t. keys. Maybe we shouldn't even give such suggestions, and rather just describe the problem of ambiguous keys being used due to the presence of an `_id` field
   



##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,156 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Use fat jar so we don't need to create a user-jar. -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-sql-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<profiles>
+		<profile>
+			<id>run-end-to-end-tests</id>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<executions>
+							<execution>
+								<id>end-to-end-tests</id>
+								<phase>integration-test</phase>
+								<goals>
+									<goal>test</goal>
+								</goals>
+								<configuration>
+									<includes>
+										<include>**/*.*</include>
+									</includes>
+									<forkCount>1</forkCount>

Review Comment:
   The comment should only be removed if this line (you know, the one it documents) is removed (which it should be!)



##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,117 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!--using mongodb shade jar to execute end-to-end test-->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-sql-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-deploy-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>

Review Comment:
   This needs to be setup similarly to https://github.com/apache/flink-connector-elasticsearch/blob/main/flink-connector-elasticsearch-e2e-tests/pom.xml#L44; a dedicated profile that activates the test.
   (This is because these tests need some prep work beforehand, specifically downloading a Flink binary)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010756431


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom

Review Comment:
   For user define serialization cases,  for example, convert data in JSON format into MongoDB.
   And the open method is just like open method in UDF that can be used to access additional features such as registering user metrics.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011340413


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContextImpl;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible to write records in a MongoDB collection.
+ *
+ * @param <IN> The type of the input elements.
+ */
+@Internal
+public class MongoWriter<IN> implements SinkWriter<IN> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriter.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+    private final MongoSinkContext sinkContext;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean flushOnCheckpoint;
+    private final List<WriteModel<BsonDocument>> bulkRequests = new ArrayList<>();
+    private final Collector<WriteModel<BsonDocument>> collector;
+    private final MongoClient mongoClient;
+
+    private boolean checkpointInProgress = false;
+    private volatile long lastSendTime = 0L;
+    private volatile long ackTime = Long.MAX_VALUE;
+
+    public MongoWriter(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            boolean flushOnCheckpoint,
+            Sink.InitContext initContext,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);
+        this.flushOnCheckpoint = flushOnCheckpoint;
+
+        checkNotNull(initContext);
+        this.mailboxExecutor = checkNotNull(initContext.getMailboxExecutor());
+
+        SinkWriterMetricGroup metricGroup = checkNotNull(initContext.metricGroup());
+        metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+
+        this.collector =
+                new CountingCollector<>(
+                        new ListCollector<>(this.bulkRequests),
+                        metricGroup.getNumRecordsSendCounter());
+
+        // Initialize the serialization schema.
+        this.sinkContext = new MongoSinkContextImpl(initContext, writeOptions);
+        try {
+            SerializationSchema.InitializationContext initializationContext =
+                    initContext.asSerializationSchemaInitializationContext();
+            serializationSchema.open(initializationContext, sinkContext, writeOptions);
+        } catch (Exception e) {
+            throw new FlinkRuntimeException("Failed to open the MongoEmitter", e);
+        }
+
+        // Initialize the mongo client.
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    @Override
+    public void write(IN element, Context context) throws IOException, InterruptedException {
+        // do not allow new bulk writes until all actions are flushed
+        while (checkpointInProgress) {
+            mailboxExecutor.yield();
+        }
+        collector.collect(serializationSchema.serialize(element, sinkContext));
+        if (isOverMaxActionsLimit() || isOverMaxFlushIntervalLimit()) {
+            doBulkWrite();
+        }
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException {
+        checkpointInProgress = true;
+        while (!bulkRequests.isEmpty() && (flushOnCheckpoint || endOfInput)) {
+            doBulkWrite();
+        }
+        checkpointInProgress = false;
+    }
+
+    @Override
+    public void close() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @VisibleForTesting
+    void doBulkWrite() throws IOException {
+        if (bulkRequests.isEmpty()) {
+            // no records to write
+            return;
+        }
+
+        int maxRetryTimes = writeOptions.getMaxRetryTimes();
+        for (int i = 0; i <= maxRetryTimes; i++) {
+            try {
+                lastSendTime = System.currentTimeMillis();
+                mongoClient
+                        .getDatabase(connectionOptions.getDatabase())
+                        .getCollection(connectionOptions.getCollection(), BsonDocument.class)
+                        .bulkWrite(bulkRequests);
+                ackTime = System.currentTimeMillis();
+                bulkRequests.clear();
+                break;
+            } catch (MongoException e) {
+                LOG.error("Bulk Write to MongoDB failed, retry times = {}", i, e);
+                if (i >= maxRetryTimes) {
+                    throw new IOException(e);
+                }
+                try {
+                    Thread.sleep(1000L * i);

Review Comment:
   Add the configuration `sink.retry.interval` and fixed the sleep time for the first attempt.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011346508


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SINK_MAX_RETRIES;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The configured class for Mongo sink. */
+@PublicEvolving
+public class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final @Nullable Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private @Nullable Integer parallelism;
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkState(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkState(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0, "The max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#NONE}

Review Comment:
   Sorry that's a typo in javadocs. The default guarantee is at-least-once.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017041964


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue() == 1;
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue() == 1L;
+        }
+        if (bsonValue.isString()) {
+            return Boolean.parseBoolean(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to boolean from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static byte convertToTinyInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (byte) 1 : (byte) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (byte) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (byte) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Byte.parseByte(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to tinyint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static short convertToSmallInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (short) 1 : (short) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (short) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (short) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Short.parseShort(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to smallint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static int convertToInt(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().intValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.intValue();
+            } else if (decimal128Value.isNegative()) {
+                return Integer.MIN_VALUE;
+            } else {
+                return Integer.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isDateTime()) {
+            return (int) Instant.ofEpochMilli(bsonValue.asDateTime().getValue()).getEpochSecond();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime();
+        }
+        if (bsonValue.isString()) {
+            return Integer.parseInt(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to integer from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static long convertToLong(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().longValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.longValue();
+            } else if (decimal128Value.isNegative()) {
+                return Long.MIN_VALUE;
+            } else {
+                return Long.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1L : 0L;
+        }
+        if (bsonValue.isDateTime()) {
+            return bsonValue.asDateTime().getValue();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime() * 1000L;
+        }
+        if (bsonValue.isString()) {
+            return Long.parseLong(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to long from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static double convertToDouble(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().doubleValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.doubleValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Double.MAX_VALUE;
+            } else {
+                return Double.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isString()) {
+            return Double.parseDouble(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to double from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static float convertToFloat(BsonValue bsonValue) {
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {

Review Comment:
   The reason why there are some loose conversions here is that MongoDB is a schema-less database and doesn't have fixed type constraints (a field can be any type), so we made some compatible conversions to make it work as well as possible in flink sql.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1038044918


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+@ExtendWith(TestLoggerExtension.class)
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);

Review Comment:
   We use a sharded cluster container to cover `sharded` split strategy.
   Now there is no need for any mocking.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032983609


##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.MongoDBContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+class MongoE2ECase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    private static final String MONGO_4_0 = "mongo:4.0.10";

Review Comment:
   In the previous modification, we ignored the test-jar compilation of `flink-connector-mongodb`. 
   In order to reuse `MongoTestUtil` in `e2e-tests` module, do we need to compile test-jar?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1096019431


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.enumerator.splitter.MongoSplitters;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSplitAssigner implements MongoSplitAssigner {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSplitAssigner.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+
+    private final LinkedList<String> remainingCollections;
+    private final List<String> alreadyProcessedCollections;
+    private final List<MongoScanSourceSplit> remainingScanSplits;
+    private final Map<String, MongoScanSourceSplit> assignedScanSplits;
+    private boolean initialized;
+
+    private transient MongoSplitters mongoSplitters;
+
+    public MongoScanSplitAssigner(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            MongoSourceEnumState sourceEnumState) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.remainingCollections = new LinkedList<>(sourceEnumState.getRemainingCollections());

Review Comment:
   The `remainingCollections` is indeed a queue semantics (not a deque), storing collections that need to be split.
   In order to simplify the serialization of the state, it is declared as a `List`.
   Do we need to explicitly declare it as a `Queue`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1096032548


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoJsonDeserializationSchema.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.reader.deserializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import org.bson.BsonDocument;
+import org.bson.json.JsonMode;
+
+import java.util.Optional;
+
+/**
+ * A schema bridge for deserializing the MongoDB's {@code BsonDocument} to MongoDB's {@link
+ * JsonMode}'s RELAXED Json string.
+ */
+@PublicEvolving
+public class MongoJsonDeserializationSchema implements MongoDeserializationSchema<String> {

Review Comment:
   Moved it into MongoSourceITCase.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1096043565


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoScanSplitAssigner.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.enumerator.splitter.MongoSplitters;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSplitAssigner implements MongoSplitAssigner {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSplitAssigner.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+
+    private final LinkedList<String> remainingCollections;
+    private final List<String> alreadyProcessedCollections;
+    private final List<MongoScanSourceSplit> remainingScanSplits;
+    private final Map<String, MongoScanSourceSplit> assignedScanSplits;
+    private boolean initialized;
+
+    private transient MongoSplitters mongoSplitters;
+
+    public MongoScanSplitAssigner(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            MongoSourceEnumState sourceEnumState) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.remainingCollections = new LinkedList<>(sourceEnumState.getRemainingCollections());
+        this.alreadyProcessedCollections = sourceEnumState.getAlreadyProcessedCollections();
+        this.remainingScanSplits = sourceEnumState.getRemainingScanSplits();
+        this.assignedScanSplits = sourceEnumState.getAssignedScanSplits();
+        this.initialized = sourceEnumState.isInitialized();
+    }
+
+    @Override
+    public void open() {
+        LOG.info("Mongo scan split assigner is opening.");
+        if (!initialized) {
+            String collectionId =
+                    String.format(
+                            "%s.%s",
+                            connectionOptions.getDatabase(), connectionOptions.getCollection());
+            remainingCollections.add(collectionId);
+            mongoSplitters = new MongoSplitters(connectionOptions, readOptions);
+            initialized = true;
+        }
+    }
+
+    @Override
+    public Optional<MongoSourceSplit> getNext() {
+        if (!remainingScanSplits.isEmpty()) {
+            // return remaining splits firstly
+            Iterator<MongoScanSourceSplit> iterator = remainingScanSplits.iterator();
+            MongoScanSourceSplit split = iterator.next();
+            iterator.remove();
+            assignedScanSplits.put(split.splitId(), split);
+            return Optional.of(split);
+        } else {
+            // it's turn for next collection
+            String nextCollection = remainingCollections.pollFirst();
+            if (nextCollection != null) {
+                // split the given collection into chunks (scan splits)
+                Collection<MongoScanSourceSplit> splits =
+                        mongoSplitters.split(new MongoNamespace(nextCollection));
+                remainingScanSplits.addAll(splits);
+                alreadyProcessedCollections.add(nextCollection);
+                return getNext();
+            } else {
+                return Optional.empty();
+            }
+        }
+    }
+
+    @Override
+    public void addSplitsBack(Collection<MongoSourceSplit> splits) {
+        for (MongoSourceSplit split : splits) {
+            if (split instanceof MongoScanSourceSplit) {
+                remainingScanSplits.add((MongoScanSourceSplit) split);
+                // we should remove the add-backed splits from the assigned list,
+                // because they are failed
+                assignedScanSplits.remove(split.splitId());
+            }
+        }
+    }
+
+    @Override
+    public MongoSourceEnumState snapshotState(long checkpointId) {
+        return new MongoSourceEnumState(
+                remainingCollections,
+                alreadyProcessedCollections,
+                remainingScanSplits,
+                assignedScanSplits,
+                initialized);
+    }
+
+    @Override
+    public boolean noMoreSplits() {
+        return initialized && remainingCollections.isEmpty() && remainingScanSplits.isEmpty();
+    }

Review Comment:
   Precondition added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1095888644


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/RowDataToBsonConverters.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonNull;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.json.JsonParseException;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+
+/** Tool class used to convert from {@link RowData} to {@link BsonValue}. */
+@Internal
+public class RowDataToBsonConverters {
+
+    // --------------------------------------------------------------------------------
+    // Runtime Converters
+    // --------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts objects of Flink Table & SQL internal data structures to
+     * corresponding {@link BsonValue} data structures.
+     */
+    @FunctionalInterface
+    public interface RowDataToBsonConverter extends Serializable {
+        BsonValue convert(Object value);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    public static RowDataToBsonConverter createConverter(LogicalType type) {

Review Comment:
   ~~This method  is also used by `MongoKeyExtractor` to convert the primary key's row type to bson type.
   But when the primary key is not composite, LogicalType may not be RowType.
   Is there any good way to differentiate?~~
   
   Fixed and expose a new method for `MongoKeyExtractor`.
   ```java
   public static SerializableFunction<Object, BsonValue> createFieldDataConverter(LogicalType type)
   ```
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097194949


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.source.enumerator.MongoSourceEnumState;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoSourceSplit}. */
+@Internal
+public interface MongoSplitAssigner extends Serializable {
+
+    /**
+     * Called to open the assigner to acquire any resources, like threads or network connections.
+     */
+    void open();
+
+    /**
+     * Called to close the assigner, in case it holds on to any resources, like threads or network
+     * connections.
+     */
+    void close() throws IOException;
+
+    /** Gets the next split. */
+    Optional<MongoSourceSplit> getNext();

Review Comment:
   > Document that. Specifically it's relationship to `noMoreSplits()`.
   
   Add document : `Gets the next split to assign to MongoSourceSplitReader when MongoSourceEnumerator receives a split request, until there are noMoreSplits().`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025246587


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+@ExtendWith(TestLoggerExtension.class)
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+    }
+
+    @Test
+    public void testShardedSplitter() {
+        MongoNamespace namespace = new MongoNamespace("test_db.test_coll");
+        BsonDocument mockCollectionMetadata = mockCollectionMetadata();
+        ArrayList<BsonDocument> mockChunksData = mockChunksData();
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(
+                        MongoReadOptions.builder().build(),
+                        mongoClient,
+                        namespace,
+                        mockCollStats());
+
+        List<MongoScanSourceSplit> expected = new ArrayList<>();
+        for (int i = 0; i < mockChunksData.size(); i++) {
+            BsonDocument mockChunkData = mockChunksData.get(i);
+            expected.add(
+                    new MongoScanSourceSplit(
+                            String.format("%s_%d", namespace, i),
+                            namespace.getDatabaseName(),
+                            namespace.getCollectionName(),
+                            mockChunkData.getDocument(MIN_FIELD),
+                            mockChunkData.getDocument(MAX_FIELD),
+                            mockCollectionMetadata.getDocument(KEY_FIELD)));
+        }
+
+        try (MockedStatic<MongoUtils> util = mockStatic(MongoUtils.class)) {
+            util.when(() -> MongoUtils.readCollectionMetadata(any(), any()))
+                    .thenReturn(mockCollectionMetadata);
+
+            util.when(() -> MongoUtils.readChunks(any(), any())).thenReturn(mockChunksData);
+
+            util.when(() -> MongoUtils.isValidShardedCollection(any())).thenReturn(true);
+
+            Collection<MongoScanSourceSplit> actual =
+                    MongoShardedSplitter.INSTANCE.split(splitContext);
+            assertThat(actual, equalTo(expected));
+        }
+    }
+
+    private BsonDocument mockCollectionMetadata() {
+        return new BsonDocument()
+                .append(ID_FIELD, new BsonObjectId())
+                .append(UUID_FIELD, new BsonBinary(UUID.randomUUID()))
+                .append(DROPPED_FIELD, BsonBoolean.FALSE)
+                .append(KEY_FIELD, ID_HINT);
+    }
+
+    private ArrayList<BsonDocument> mockChunksData() {
+        ArrayList<BsonDocument> chunks = new ArrayList<>();
+        chunks.add(mockChunkData(1));
+        chunks.add(mockChunkData(2));
+        chunks.add(mockChunkData(3));
+        return chunks;

Review Comment:
   ```suggestion
           return Arrays.asList(...);
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+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.SINK_MAX_RETRIES;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int batchSize;
+    private final long batchIntervalMs;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    private MongoWriteOptions(
+            int batchSize,
+            long batchIntervalMs,
+            int maxRetries,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee) {
+        this.batchSize = batchSize;
+        this.batchIntervalMs = batchIntervalMs;
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+    }
+
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    public long getBatchIntervalMs() {
+        return batchIntervalMs;
+    }
+
+    public int getMaxRetries() {
+        return maxRetries;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return batchSize == that.batchSize
+                && batchIntervalMs == that.batchIntervalMs
+                && maxRetries == that.maxRetries
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                batchSize, batchIntervalMs, maxRetries, retryIntervalMs, deliveryGuarantee);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int batchSize = BUFFER_FLUSH_MAX_ROWS.defaultValue();
+        private long batchIntervalMs = BUFFER_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetries = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param batchSize the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBatchSize(int batchSize) {
+            checkArgument(
+                    batchSize == -1 || batchSize > 0,
+                    "Max number of batch size must be larger than 0.");
+            this.batchSize = batchSize;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param batchIntervalMs the batch flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBatchIntervalMs(long batchIntervalMs) {
+            checkArgument(
+                    batchIntervalMs == -1 || batchIntervalMs >= 0,
+                    "The batch flush interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");

Review Comment:
   What does it mean for the interval to be 0 (opposed to -1)?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/MongoSink.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.api.java.ClosureCleaner;
+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.sink.writer.MongoWriter;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+
+import com.mongodb.client.model.WriteModel;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Mongo sink converts each incoming element into MongoDB {@link WriteModel} (bulk write action) and
+ * bulk writes to mongodb when the number of actions is greater than batchSize or the flush interval
+ * is greater than batchIntervalMs.
+ *
+ * <p>The following example shows how to create a MongoSink receiving records of {@code Document}

Review Comment:
   ```suggestion
    * <p>The following example shows how to create a MongoSink sending records of {@code Document}
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+@ExtendWith(TestLoggerExtension.class)
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);

Review Comment:
   It's a bad sign when you have to mock things; that should only be done when there is a _very_ hard requirement, as per the Flink code quality guide.
   
   Consider refactoring the `MongoSplitContext` to not use the `MongoClient` directly, but hide it behind a `Provider<MongoCollection<BsonDocument>` / `Function<MongoNamespace, MongoCollection<BsonDocument>>` / `BiFunction<String, String, MongoCollection<BsonDocument>>`.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+@ExtendWith(TestLoggerExtension.class)
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+    }
+
+    @Test
+    public void testShardedSplitter() {
+        MongoNamespace namespace = new MongoNamespace("test_db.test_coll");
+        BsonDocument mockCollectionMetadata = mockCollectionMetadata();
+        ArrayList<BsonDocument> mockChunksData = mockChunksData();

Review Comment:
   ```suggestion
           List<BsonDocument> mockChunksData = mockChunksData();
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializerTest.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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;
+
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonMaxKey;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+import static org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumStateSerializer.INSTANCE;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+
+/** Unit tests for {@link MongoSourceEnumStateSerializer}. */
+@ExtendWith(TestLoggerExtension.class)
+public class MongoSourceEnumStateSerializerTest {
+
+    @Test
+    void serializeAndDeserializeMongoSourceEnumState() throws Exception {
+        boolean initialized = false;
+        List<String> remainingCollections = Arrays.asList("db.remains0", "db.remains1");
+        List<String> alreadyProcessedCollections = Arrays.asList("db.processed0", "db.processed1");
+        List<MongoScanSourceSplit> remainingScanSplits = new ArrayList<>();
+        remainingScanSplits.add(createSourceSplit(0));
+        remainingScanSplits.add(createSourceSplit(1));

Review Comment:
   Arrays.asList



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reversedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // It behaves as append-only when no primary key is declared and no reversed _id is present.
+        // We use anonymous classes instead of lambdas for a reason here. It is
+        // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas
+        // (MSHADE-260).
+        if (!primaryKey.isPresent() && !reversedId.isPresent()) {
+            return new SerializableFunction<RowData, BsonValue>() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public BsonValue apply(RowData rowData) {
+                    return null;
+                }
+            };

Review Comment:
   This could even be a singleton.
   
   Does the problem still exist when it is a method reference to a static method?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+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.SINK_MAX_RETRIES;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int batchSize;
+    private final long batchIntervalMs;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    private MongoWriteOptions(
+            int batchSize,
+            long batchIntervalMs,
+            int maxRetries,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee) {
+        this.batchSize = batchSize;
+        this.batchIntervalMs = batchIntervalMs;
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+    }
+
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    public long getBatchIntervalMs() {
+        return batchIntervalMs;
+    }

Review Comment:
   Consider having this return an Optional to better communicate whether it is enabled or not.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reversedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // It behaves as append-only when no primary key is declared and no reversed _id is present.
+        // We use anonymous classes instead of lambdas for a reason here. It is
+        // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas
+        // (MSHADE-260).
+        if (!primaryKey.isPresent() && !reversedId.isPresent()) {
+            return new SerializableFunction<RowData, BsonValue>() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public BsonValue apply(RowData rowData) {
+                    return null;
+                }
+            };

Review Comment:
   Let's create a dedicated class and add a comment to that instead.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reversedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // It behaves as append-only when no primary key is declared and no reversed _id is present.
+        // We use anonymous classes instead of lambdas for a reason here. It is
+        // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas
+        // (MSHADE-260).
+        if (!primaryKey.isPresent() && !reversedId.isPresent()) {
+            return new SerializableFunction<RowData, BsonValue>() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public BsonValue apply(RowData rowData) {
+                    return null;
+                }
+            };
+        }
+
+        if (reversedId.isPresent()) {
+            // Primary key should be declared as (_id) when the mongo reversed _id is present.

Review Comment:
   typo: Reversed -> Reserved



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+@ExtendWith(TestLoggerExtension.class)
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+    }
+
+    @Test
+    public void testShardedSplitter() {
+        MongoNamespace namespace = new MongoNamespace("test_db.test_coll");
+        BsonDocument mockCollectionMetadata = mockCollectionMetadata();
+        ArrayList<BsonDocument> mockChunksData = mockChunksData();
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(
+                        MongoReadOptions.builder().build(),
+                        mongoClient,
+                        namespace,
+                        mockCollStats());
+
+        List<MongoScanSourceSplit> expected = new ArrayList<>();
+        for (int i = 0; i < mockChunksData.size(); i++) {
+            BsonDocument mockChunkData = mockChunksData.get(i);
+            expected.add(
+                    new MongoScanSourceSplit(
+                            String.format("%s_%d", namespace, i),
+                            namespace.getDatabaseName(),
+                            namespace.getCollectionName(),
+                            mockChunkData.getDocument(MIN_FIELD),
+                            mockChunkData.getDocument(MAX_FIELD),
+                            mockCollectionMetadata.getDocument(KEY_FIELD)));
+        }
+
+        try (MockedStatic<MongoUtils> util = mockStatic(MongoUtils.class)) {

Review Comment:
   see above about mocking; this will require a bit of refactoring. Probably a sort of wrapper around the `MongoUtils`.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+@ExtendWith(TestLoggerExtension.class)
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+    }
+
+    @Test
+    public void testShardedSplitter() {
+        MongoNamespace namespace = new MongoNamespace("test_db.test_coll");
+        BsonDocument mockCollectionMetadata = mockCollectionMetadata();
+        ArrayList<BsonDocument> mockChunksData = mockChunksData();
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(
+                        MongoReadOptions.builder().build(),
+                        mongoClient,
+                        namespace,
+                        mockCollStats());
+
+        List<MongoScanSourceSplit> expected = new ArrayList<>();
+        for (int i = 0; i < mockChunksData.size(); i++) {
+            BsonDocument mockChunkData = mockChunksData.get(i);
+            expected.add(
+                    new MongoScanSourceSplit(
+                            String.format("%s_%d", namespace, i),
+                            namespace.getDatabaseName(),
+                            namespace.getCollectionName(),
+                            mockChunkData.getDocument(MIN_FIELD),
+                            mockChunkData.getDocument(MAX_FIELD),
+                            mockCollectionMetadata.getDocument(KEY_FIELD)));
+        }
+
+        try (MockedStatic<MongoUtils> util = mockStatic(MongoUtils.class)) {
+            util.when(() -> MongoUtils.readCollectionMetadata(any(), any()))
+                    .thenReturn(mockCollectionMetadata);
+
+            util.when(() -> MongoUtils.readChunks(any(), any())).thenReturn(mockChunksData);
+
+            util.when(() -> MongoUtils.isValidShardedCollection(any())).thenReturn(true);
+
+            Collection<MongoScanSourceSplit> actual =
+                    MongoShardedSplitter.INSTANCE.split(splitContext);
+            assertThat(actual, equalTo(expected));
+        }
+    }
+
+    private BsonDocument mockCollectionMetadata() {
+        return new BsonDocument()
+                .append(ID_FIELD, new BsonObjectId())
+                .append(UUID_FIELD, new BsonBinary(UUID.randomUUID()))
+                .append(DROPPED_FIELD, BsonBoolean.FALSE)
+                .append(KEY_FIELD, ID_HINT);
+    }
+
+    private ArrayList<BsonDocument> mockChunksData() {
+        ArrayList<BsonDocument> chunks = new ArrayList<>();
+        chunks.add(mockChunkData(1));
+        chunks.add(mockChunkData(2));
+        chunks.add(mockChunkData(3));
+        return chunks;
+    }
+
+    private BsonDocument mockChunkData(int index) {
+        return new BsonDocument()
+                .append(MIN_FIELD, new BsonDocument(ID_FIELD, new BsonInt32(index * 100)))
+                .append(MAX_FIELD, new BsonDocument(ID_FIELD, new BsonInt32((index + 1) * 100)))
+                .append(SHARD_FIELD, new BsonString("shard-" + index));
+    }
+
+    private BsonDocument mockCollStats() {

Review Comment:
   ```suggestion
       private static BsonDocument mockCollStats() {
   ```
   add static modifier when possible



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reversedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // It behaves as append-only when no primary key is declared and no reversed _id is present.
+        // We use anonymous classes instead of lambdas for a reason here. It is
+        // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas
+        // (MSHADE-260).
+        if (!primaryKey.isPresent() && !reversedId.isPresent()) {
+            return new SerializableFunction<RowData, BsonValue>() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public BsonValue apply(RowData rowData) {
+                    return null;
+                }
+            };
+        }
+
+        if (reversedId.isPresent()) {
+            // Primary key should be declared as (_id) when the mongo reversed _id is present.
+            if (!primaryKey.isPresent()
+                    || isCompoundPrimaryKey(primaryKeyIndexes)
+                    || !primaryKeyContainsReversedId(primaryKey.get())) {
+                throw new IllegalArgumentException(
+                        "The primary key should be declared as (_id) when mongo reversed _id field is present");

Review Comment:
   Should it _really_ be declared as (_id), or maybe as something completely different? is `(_id)` some special thing?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";

Review Comment:
   Would it make sense to somehow link this to `MongoConstants#ID_FIELD`?



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/common/utils/MongoSerdeUtilsTest.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.util.TestLoggerExtension;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+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.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Unit tests for {@link MongoSerdeUtils}. */
+@ExtendWith(TestLoggerExtension.class)

Review Comment:
   Instead of adding this to every test copy this file: https://github.com/apache/flink/blob/master/flink-core/src/test/resources/META-INF/services/org.junit.jupiter.api.extension.Extension to the test/resources/META-INF/services



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+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.MongoReadOptions;
+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;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
+import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** A {@link DynamicTableSource} for MongoDB. */
+@Internal
+public class MongoDynamicTableSource
+        implements ScanTableSource,
+                LookupTableSource,
+                SupportsProjectionPushDown,
+                SupportsLimitPushDown {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    @Nullable private final LookupCache lookupCache;
+    private final int lookupMaxRetries;
+    private final long lookupRetryIntervalMs;
+    private DataType physicalRowDataType;
+    private int limit = -1;

Review Comment:
   effectively unused; let's remove it



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable

Review Comment:
   Why can this return null? Does that happen if the database/collection does not exist, or...?
   Consider returning an optional instead.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reversedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // It behaves as append-only when no primary key is declared and no reversed _id is present.
+        // We use anonymous classes instead of lambdas for a reason here. It is
+        // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas
+        // (MSHADE-260).
+        if (!primaryKey.isPresent() && !reversedId.isPresent()) {
+            return new SerializableFunction<RowData, BsonValue>() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public BsonValue apply(RowData rowData) {
+                    return null;
+                }
+            };
+        }
+
+        if (reversedId.isPresent()) {
+            // Primary key should be declared as (_id) when the mongo reversed _id is present.
+            if (!primaryKey.isPresent()
+                    || isCompoundPrimaryKey(primaryKeyIndexes)
+                    || !primaryKeyContainsReversedId(primaryKey.get())) {
+                throw new IllegalArgumentException(
+                        "The primary key should be declared as (_id) when mongo reversed _id field is present");
+            }
+        }
+
+        DataType primaryKeyType;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            DataType physicalRowDataType = resolvedSchema.toPhysicalRowDataType();
+            primaryKeyType = Projection.of(primaryKeyIndexes).project(physicalRowDataType);
+        } else {
+            int primaryKeyIndex = primaryKeyIndexes[0];
+            Optional<Column> column = resolvedSchema.getColumn(primaryKeyIndex);
+            if (!column.isPresent()) {
+                throw new IllegalStateException(
+                        String.format(
+                                "No primary key column found with index '%s'.", primaryKeyIndex));
+            }
+            primaryKeyType = column.get().getDataType();
+        }
+
+        MongoValidationUtils.validatePrimaryKey(primaryKeyType);
+
+        return new MongoKeyExtractor(primaryKeyType.getLogicalType(), primaryKeyIndexes);
+    }
+
+    private static boolean isCompoundPrimaryKey(int[] primaryKeyIndexes) {
+        return primaryKeyIndexes.length > 1;
+    }
+
+    private static boolean primaryKeyContainsReversedId(UniqueConstraint primaryKey) {

Review Comment:
   typo: Reversed -> Reserved



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025027441


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable
+    public static BsonDocument readCollectionMetadata(
+            MongoClient mongoClient, MongoNamespace namespace) {
+        MongoCollection<BsonDocument> collection =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(COLLECTIONS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        return collection
+                .find(eq(ID_FIELD, namespace.getFullName()))
+                .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD))
+                .first();
+    }
+
+    public static boolean isValidShardedCollection(BsonDocument collectionMetadata) {
+        return collectionMetadata != null
+                && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue();
+    }
+
+    public static List<BsonDocument> readChunks(
+            MongoClient mongoClient, BsonDocument collectionMetadata) {
+        MongoCollection<BsonDocument> chunks =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(CHUNKS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        Bson filter =
+                or(
+                        new BsonDocument(NAMESPACE_FIELD, collectionMetadata.get(ID_FIELD)),
+                        // MongoDB 4.9.0 removed ns field of config.chunks collection, using

Review Comment:
   This raises an interesting question on compatibility. Which versions do we intend to support / which are still actively maintained by MongoDB?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1022827705


##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,157 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!--using mongodb shade jar to execute end-to-end test-->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-sql-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<profiles>
+		<profile>
+			<id>run-end-to-end-tests</id>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<executions>
+							<execution>
+								<id>end-to-end-tests</id>
+								<phase>integration-test</phase>
+								<goals>
+									<goal>test</goal>
+								</goals>
+								<configuration>
+									<includes>
+										<include>**/*.*</include>
+									</includes>
+									<!-- E2E tests must not access flink-dist concurrently. -->
+									<forkCount>1</forkCount>
+									<systemPropertyVariables>
+										<moduleDir>${project.basedir}</moduleDir>
+									</systemPropertyVariables>
+								</configuration>
+							</execution>
+						</executions>
+					</plugin>
+				</plugins>
+			</build>
+		</profile>
+	</profiles>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-deploy-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>default-test</id>
+						<phase>none</phase>
+					</execution>
+					<execution>
+						<id>integration-tests</id>
+						<phase>none</phase>
+					</execution>
+				</executions>
+			</plugin>

Review Comment:
   How expensive are the e2e tests? I think we could just run them as part of the integration-tests phase.
   (== remove/inline run-end-to-end-tests profile)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1026099789


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reversedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // It behaves as append-only when no primary key is declared and no reversed _id is present.
+        // We use anonymous classes instead of lambdas for a reason here. It is
+        // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas
+        // (MSHADE-260).
+        if (!primaryKey.isPresent() && !reversedId.isPresent()) {
+            return new SerializableFunction<RowData, BsonValue>() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public BsonValue apply(RowData rowData) {
+                    return null;
+                }
+            };
+        }
+
+        if (reversedId.isPresent()) {
+            // Primary key should be declared as (_id) when the mongo reversed _id is present.
+            if (!primaryKey.isPresent()
+                    || isCompoundPrimaryKey(primaryKeyIndexes)
+                    || !primaryKeyContainsReversedId(primaryKey.get())) {
+                throw new IllegalArgumentException(
+                        "The primary key should be declared as (_id) when mongo reversed _id field is present");

Review Comment:
   The MongoDB's `_id` is similar to Elasticsearch's `_id`.
   If `_id` appears in RowData, but the user declares other keys as primary keys, some ambiguity will arise:
   Should the _id of RowData be directly used as `_id` or other keys as `_id`?
   
   ```sql
   CREATE TABLE t (
       _id String,  -- shall we ignore this field in this case?
       f0  String,
       f1  String,
       PRIMARY KEY (f0, f1) NOT ENFORCED
   )
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1096065712


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonDocument;
+import org.bson.BsonType;
+import org.bson.BsonValue;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.DEFAULT_JSON_WRITER_SETTINGS;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {

Review Comment:
   Thanks, a more explicit statement is needed here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1096990499


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriterITCase.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.connector.mongodb.testutils.MongoTestUtil;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.DeleteOneModel;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.UpdateOneModel;
+import com.mongodb.client.model.UpdateOptions;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+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.util.Optional;
+import java.util.OptionalLong;
+
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreNotWritten;
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link MongoWriter}. */
+@Testcontainers
+public class MongoWriterITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriterITCase.class);
+
+    private static final String TEST_DATABASE = "test_writer";
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    private static MongoClient mongoClient;
+    private static MetricListener metricListener;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void setUp() {
+        metricListener = new MetricListener();
+    }
+
+    @Test
+    void testWriteOnBulkFlush() throws Exception {
+        final String collection = "test-bulk-flush-without-checkpoint";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 5;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+
+            // Ignore flush on checkpoint
+            writer.flush(false);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3, 4);
+
+            // Trigger flush
+            writer.write(buildMessage(5), null);
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5);
+
+            writer.write(buildMessage(6), null);
+            assertThatIdsAreNotWritten(collectionOf(collection), 6);
+
+            // Force flush
+            writer.doBulkWrite();
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5, 6);
+        }
+    }
+
+    @Test
+    void testWriteOnBatchIntervalFlush() throws Exception {
+        final String collection = "test-bulk-flush-with-interval";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = -1;
+        final int batchIntervalMs = 1000;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+            writer.doBulkWrite();
+        }
+
+        assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4);
+    }
+
+    @Test
+    void testWriteOnCheckpoint() throws Exception {
+        final String collection = "test-bulk-flush-with-checkpoint";
+        final boolean flushOnCheckpoint = true;
+        final int batchSize = -1;
+        final int batchIntervalMs = -1;
+
+        // Enable flush on checkpoint
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3);
+
+            // Trigger flush
+            writer.flush(false);
+
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3);
+        }
+    }
+
+    @Test
+    void testIncrementRecordsSendMetric() throws Exception {
+        final String collection = "test-inc-records-send";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Counter> recordsSend =
+                    metricListener.getCounter(MetricNames.NUM_RECORDS_SEND);
+            writer.write(buildMessage(1), null);
+            // Update existing index
+            writer.write(buildMessage(2, "u"), null);
+            // Delete index
+            writer.write(buildMessage(3, "d"), null);
+
+            writer.doBulkWrite();
+
+            assertThat(recordsSend.isPresent()).isTrue();
+            assertThat(recordsSend.get().getCount()).isEqualTo(3L);
+        }
+    }
+
+    @Test
+    void testCurrentSendTime() throws Exception {
+        final String collection = "test-current-send-time";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Gauge<Long>> currentSendTime =
+                    metricListener.getGauge("currentSendTime");
+
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+
+            writer.doBulkWrite();
+
+            assertThat(currentSendTime.isPresent()).isTrue();
+            assertThat(currentSendTime.get().getValue()).isGreaterThan(0L);

Review Comment:
   `currentSendTime = ackTime - lastSendTime;`
   There is a network IO operation, and under normal circumstances, it will be greater than 1ms.
   Is there a better way to test it?
   
   ```java
                   lastSendTime = System.currentTimeMillis();
                   mongoClient
                           .getDatabase(connectionOptions.getDatabase())
                           .getCollection(connectionOptions.getCollection(), BsonDocument.class)
                           .bulkWrite(bulkRequests);
                   ackTime = System.currentTimeMillis();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1095888644


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/RowDataToBsonConverters.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonNull;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.json.JsonParseException;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+
+/** Tool class used to convert from {@link RowData} to {@link BsonValue}. */
+@Internal
+public class RowDataToBsonConverters {
+
+    // --------------------------------------------------------------------------------
+    // Runtime Converters
+    // --------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts objects of Flink Table & SQL internal data structures to
+     * corresponding {@link BsonValue} data structures.
+     */
+    @FunctionalInterface
+    public interface RowDataToBsonConverter extends Serializable {
+        BsonValue convert(Object value);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    public static RowDataToBsonConverter createConverter(LogicalType type) {

Review Comment:
   @zentol 
   This method  is also used by `MongoKeyExtractor` to convert the primary key's row type to bson type.
   But when the primary key is not composite, LogicalType may not be RowType.
   Is there any good way to differentiate?
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1095969209


##########
flink-connector-mongodb-e2e-tests/src/test/resources/e2e_append_only.sql:
##########
@@ -0,0 +1,44 @@
+--/*
+-- * Licensed to the Apache Software Foundation (ASF) under one
+-- * or more contributor license agreements.  See the NOTICE file
+-- * distributed with this work for additional information
+-- * regarding copyright ownership.  The ASF licenses this file
+-- * to you under the Apache License, Version 2.0 (the
+-- * "License"); you may not use this file except in compliance
+-- * with the License.  You may obtain a copy of the License at
+-- *
+-- *     http://www.apache.org/licenses/LICENSE-2.0
+-- *
+-- * Unless required by applicable law or agreed to in writing, software
+-- * distributed under the License is distributed on an "AS IS" BASIS,
+-- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- * See the License for the specific language governing permissions and
+-- * limitations under the License.
+-- */
+
+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',

Review Comment:
   The ports exposed by each mongodb container are random.
   The 27017 port accessed here is the port inside the container by network aliases.
   I think that running CI at the same time may not affect each other.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097668314


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSampleSplitter.class);
+
+    public static final MongoSampleSplitter INSTANCE = new MongoSampleSplitter();
+
+    private MongoSampleSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+
+        long count = splitContext.getCount();
+        long partitionSizeInBytes = readOptions.getPartitionSize().getBytes();
+        int samplesPerPartition = readOptions.getSamplesPerPartition();
+
+        long avgObjSizeInBytes = splitContext.getAvgObjSize();
+        if (avgObjSizeInBytes == 0L) {
+            LOG.info(
+                    "{} seems to be an empty collection, Returning a single partition.", namespace);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        long numDocumentsPerPartition = partitionSizeInBytes / avgObjSizeInBytes;
+        if (numDocumentsPerPartition >= count) {
+            LOG.info(
+                    "Fewer documents ({}) than the number of documents per partition ({}), Returning a single partition.",
+                    count,
+                    numDocumentsPerPartition);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        int numberOfSamples =
+                (int) Math.ceil((samplesPerPartition * count * 1.0d) / numDocumentsPerPartition);
+
+        List<BsonDocument> samples =
+                splitContext
+                        .getMongoCollection()
+                        .aggregate(
+                                Arrays.asList(
+                                        Aggregates.sample(numberOfSamples),
+                                        Aggregates.project(Projections.include(ID_FIELD)),
+                                        Aggregates.sort(Sorts.ascending(ID_FIELD))))
+                        .allowDiskUse(true)
+                        .into(new ArrayList<>());

Review Comment:
   Thanks @zentol for this great idea.
   I have refactored the splitter using MIN to replace the first sample and MAX to replace the last sample.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1044309412


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,677 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = new Object[in.size()];
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032957642


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoShardedSplitter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.mongodb.MongoException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isShardedCollectionDropped;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readChunks;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readCollectionMetadata;
+
+/**
+ * Sharded Partitioner
+ *
+ * <p>Uses the chunks collection and partitions the collection based on the sharded collections
+ * chunk ranges.
+ *
+ * <p>The following config collections' read privilege is required.
+ *
+ * <ul>
+ *   <li>config.collections
+ *   <li>config.chunks
+ * </ul>
+ */
+@Internal
+public class MongoShardedSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+    public static final MongoShardedSplitter INSTANCE = new MongoShardedSplitter();
+
+    private MongoShardedSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoClient mongoClient = splitContext.getMongoClient();
+
+        List<BsonDocument> chunks;
+        Optional<BsonDocument> collectionMetadata;
+        try {
+            collectionMetadata = readCollectionMetadata(mongoClient, namespace);
+            if (!collectionMetadata.isPresent()) {
+                LOG.error(
+                        "Do sharded split failed, collection {} does not appear to be sharded.",
+                        namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, %s is not a sharded collection.",
+                                namespace));
+            }
+
+            if (isShardedCollectionDropped(collectionMetadata.get())) {
+                LOG.error("Do sharded split failed, collection {} was dropped.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format("Do sharded split failed, %s was dropped.", namespace));
+            }
+
+            chunks = readChunks(mongoClient, collectionMetadata.get());
+            if (chunks.isEmpty()) {
+                LOG.error("Do sharded split failed, chunks of {} is empty.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, chunks of %s is empty.", namespace));
+            }

Review Comment:
   As long as we shard the collection, even if it is empty, a record will be generated in `config.chunks`.
   ```javascript
   {
       "_id" : ObjectId("63838c89ae7bc37861d753a7"),
       "uuid" : UUID("cce0b7c9-4c67-4d01-ad1f-ddc13d91dc49"),
       "min" : {
           "user_id" : { "$minKey" : 1 },
           "product_no" : { "$minKey" : 1 },
           "product_kind" : { "$minKey" : 1 }
       },
       "max" : {
           "user_id" : { "$maxKey" : 1 },
           "product_no" : { "$maxKey" : 1 },
           "product_kind" : { "$maxKey" : 1 }
       },
       "shard" : "rs0-shard",
       "lastmod" : Timestamp(1, 0),
       "history" : [ 
           {
               "validAfter" : Timestamp(1669565577, 14),
               "shard" : "rs0-shard"
           }
       ]
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1031990672


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoJsonDeserializationSchema.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.reader.deserializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import org.bson.BsonDocument;
+import org.bson.json.JsonMode;
+
+import java.util.Optional;
+
+/**
+ * A schema bridge for deserializing the MongoDB's {@code BsonDocument} to MongoDB's {@link
+ * JsonMode}'s RELAXED Json string.
+ */
+@PublicEvolving
+public class MongoJsonDeserializationSchema implements MongoDeserializationSchema<String> {

Review Comment:
   Used to convert `BsonDocument` to [Json format](https://www.mongodb.com/docs/manual/reference/mongodb-extended-json/).
   
   ```java
       public void test() throws Exception {
           String database = "test";
           String collection = "test_coll";
   
           StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
           env.setParallelism(1);
   
           MongoSource<String> mongoSource =
                   MongoSource.<String>builder()
                           .setUri("mongodb://127.0.0.1:27017")
                           .setDatabase("test")
                           .setCollection("test")
                           .setDeserializationSchema(new MongoJsonDeserializationSchema())
                           .build();
   
           env.fromSource(mongoSource, WatermarkStrategy.noWatermarks(), "MongoSource")
                   .setParallelism(1)
                   .print();
   
           env.execute();
       }
   ```
   
   Output Example
   ```json
   {"_id": {"$oid": "63803fa3fdca9606df01da9e"}, "f0": 1, "f1": "hRKnchMIFNZMfAtBtYRViNleBezFgTSq"}
   {"_id": {"$oid": "63803fa3fdca9606df01da9f"}, "f0": 2, "f1": "oLRoMyLXvUxFwmOUINDwwhJeKpaSDJiM"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa0"}, "f0": 3, "f1": "esOnWaOwCUcsEGqisWNrupkbmIKtPZVS"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa1"}, "f0": 4, "f1": "lEqQYSNJKjUNILHFDnFQugpijBMXbqpy"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa2"}, "f0": 5, "f1": "lcgroaWbztgNIxUHwbVoVmSSLpmBYvXC"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa3"}, "f0": 6, "f1": "ELFXtzuJpLkWuIlXRVChRBqxnklbPGQF"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa4"}, "f0": 7, "f1": "WtFiNqcHCYmriyYFLBZtyqMPQDEidULJ"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa5"}, "f0": 8, "f1": "VDmoZYzuANQfpQtTnUdwXueNILPUgyNq"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa6"}, "f0": 9, "f1": "gLSwHUyzTHJjyOBrkjKlWTfsjtMeedef"}
   {"_id": {"$oid": "63803fa3fdca9606df01daa7"}, "f0": 10, "f1": "hzHIcuDdvnEHcQYsKmPkbtmmzhWIyOTu"}
   ```
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1016636631


##########
pom.xml:
##########
@@ -0,0 +1,426 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16.0</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<assertj.version>3.21.0</assertj.version>
+		<archunit.version>0.22.0</archunit.version>
+		<testcontainers.version>1.17.2</testcontainers.version>
+		<mockito.version>3.4.6</mockito.version>
+
+		<japicmp.skip>false</japicmp.skip>
+		<japicmp.referenceVersion>1.15.0</japicmp.referenceVersion>
+
+		<slf4j.version>1.7.36</slf4j.version>
+		<log4j.version>2.17.2</log4j.version>
+
+		<flink.parent.artifactId>flink-connector-mongodb-parent</flink.parent.artifactId>
+	</properties>
+
+	<modules>
+		<module>flink-connector-mongodb</module>
+		<module>flink-sql-connector-mongodb</module>
+		<module>flink-connector-mongodb-e2e-tests</module>
+	</modules>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-force-shading</artifactId>
+			<version>${flink.shaded.version}</version>
+		</dependency>
+
+		<!-- Root dependencies for all projects -->
+
+		<!-- Logging API -->
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- 'javax.annotation' classes like '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.vintage</groupId>
+			<artifactId>junit-vintage-engine</artifactId>
+			<scope>test</scope>
+		</dependency>

Review Comment:
   This shouldn't be necessary. All tests should be implemented with junit 5.



##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,117 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!--using mongodb shade jar to execute end-to-end test-->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-sql-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-deploy-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>

Review Comment:
   Why is this disabled? How would the e2e tests be run?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/config/MongoConnectionOptions.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.common.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.mongodb.ConnectionString;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The connection configuration class for MongoDB. */
+@PublicEvolving
+public class MongoConnectionOptions implements Serializable {
+
+    private final String uri;
+    private final String database;
+    private final String collection;
+
+    private MongoConnectionOptions(String uri, String database, String collection) {
+        this.uri = checkNotNull(uri);
+        this.database = checkNotNull(database);
+        this.collection = checkNotNull(collection);
+    }
+
+    public String getUri() {
+        return uri;
+    }
+
+    public String getDatabase() {
+        return database;
+    }
+
+    public String getCollection() {
+        return collection;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoConnectionOptions that = (MongoConnectionOptions) o;
+        return Objects.equals(uri, that.uri)
+                && Objects.equals(database, that.database)
+                && Objects.equals(collection, that.collection);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(uri, database, collection);
+    }
+
+    public static MongoConnectionOptionsBuilder builder() {
+        return new MongoConnectionOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoConnectionOptions}. */
+    @PublicEvolving
+    public static class MongoConnectionOptionsBuilder {
+        private String uri;
+        private String database;
+        private String collection;
+
+        /**
+         * Sets the connection string of MongoDB.
+         *
+         * @param uri connection string of MongoDB
+         * @return this builder
+         */
+        public MongoConnectionOptionsBuilder setUri(String uri) {
+            this.uri = new ConnectionString(uri).getConnectionString();
+            return this;

Review Comment:
   Do we create the `ConnectionString` for validation purposes?



##########
pom.xml:
##########
@@ -0,0 +1,426 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16.0</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<assertj.version>3.21.0</assertj.version>
+		<archunit.version>0.22.0</archunit.version>
+		<testcontainers.version>1.17.2</testcontainers.version>
+		<mockito.version>3.4.6</mockito.version>
+
+		<japicmp.skip>false</japicmp.skip>
+		<japicmp.referenceVersion>1.15.0</japicmp.referenceVersion>
+
+		<slf4j.version>1.7.36</slf4j.version>
+		<log4j.version>2.17.2</log4j.version>
+
+		<flink.parent.artifactId>flink-connector-mongodb-parent</flink.parent.artifactId>
+	</properties>
+
+	<modules>
+		<module>flink-connector-mongodb</module>
+		<module>flink-sql-connector-mongodb</module>
+		<module>flink-connector-mongodb-e2e-tests</module>
+	</modules>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-force-shading</artifactId>
+			<version>${flink.shaded.version}</version>
+		</dependency>
+
+		<!-- Root dependencies for all projects -->
+
+		<!-- Logging API -->
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- 'javax.annotation' classes like '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.vintage</groupId>
+			<artifactId>junit-vintage-engine</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.assertj</groupId>
+			<artifactId>assertj-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-inline</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-core</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Tests will have log4j as the default logging framework available -->
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-slf4j-impl</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- API bridge between log4j 1 and 2 -->
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-1.2-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-production</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.mongodb</groupId>
+				<artifactId>mongodb-driver-sync</artifactId>
+				<version>${mongodb.version}</version>
+			</dependency>
+
+			<!-- Flink dependencies -->
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-test-utils</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-test-utils-junit</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<!-- Flink ArchUnit -->
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-base</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-test</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-production</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<!-- This manages the 'javax.annotation' annotations (JSR305) -->
+			<dependency>
+				<groupId>com.google.code.findbugs</groupId>
+				<artifactId>jsr305</artifactId>
+				<version>1.3.9</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.slf4j</groupId>
+				<artifactId>slf4j-api</artifactId>
+				<version>${slf4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-slf4j-impl</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-api</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-core</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<!-- API bridge between log4j 1 and 2 -->
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-1.2-api</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<!-- For dependency convergence -->
+			<dependency>
+				<groupId>com.fasterxml.jackson</groupId>
+				<artifactId>jackson-bom</artifactId>
+				<type>pom</type>
+				<scope>import</scope>
+				<version>2.13.2.20220328</version>

Review Comment:
   ```suggestion
   				<version>2.13.4.20221013</version>
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.split.MongoScanSourceSplitReader;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The configuration class for MongoDB source. */
+@PublicEvolving
+public class MongoReadOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int fetchSize;
+
+    private final int cursorBatchSize;
+
+    private final boolean noCursorTimeout;
+
+    private final PartitionStrategy partitionStrategy;
+
+    private final MemorySize partitionSize;
+
+    private final int samplesPerPartition;
+
+    private MongoReadOptions(
+            int fetchSize,
+            int cursorBatchSize,
+            boolean noCursorTimeout,
+            PartitionStrategy partitionStrategy,
+            MemorySize partitionSize,
+            int samplesPerPartition) {
+        this.fetchSize = fetchSize;
+        this.cursorBatchSize = cursorBatchSize;
+        this.noCursorTimeout = noCursorTimeout;
+        this.partitionStrategy = partitionStrategy;
+        this.partitionSize = partitionSize;
+        this.samplesPerPartition = samplesPerPartition;
+    }
+
+    public int getFetchSize() {
+        return fetchSize;
+    }
+
+    public int getCursorBatchSize() {
+        return cursorBatchSize;
+    }
+
+    public boolean isNoCursorTimeout() {
+        return noCursorTimeout;
+    }
+
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    public MemorySize getPartitionSize() {
+        return partitionSize;
+    }
+
+    public int getSamplesPerPartition() {
+        return samplesPerPartition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoReadOptions that = (MongoReadOptions) o;
+        return cursorBatchSize == that.cursorBatchSize
+                && noCursorTimeout == that.noCursorTimeout
+                && partitionStrategy == that.partitionStrategy
+                && samplesPerPartition == that.samplesPerPartition
+                && Objects.equals(partitionSize, that.partitionSize);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                cursorBatchSize,
+                noCursorTimeout,
+                partitionStrategy,
+                partitionSize,
+                samplesPerPartition);
+    }
+
+    public static MongoReadOptionsBuilder builder() {
+        return new MongoReadOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoReadOptions}. */
+    @PublicEvolving
+    public static class MongoReadOptionsBuilder {
+        private int fetchSize = SCAN_FETCH_SIZE.defaultValue();
+        private int cursorBatchSize = SCAN_CURSOR_BATCH_SIZE.defaultValue();
+        private boolean noCursorTimeout = SCAN_CURSOR_NO_TIMEOUT.defaultValue();
+        private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue();
+        private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue();
+        private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue();
+
+        /**
+         * Sets the number of documents should be fetched per round-trip when reading.
+         *
+         * @param fetchSize the number of documents should be fetched per round-trip when reading.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setFetchSize(int fetchSize) {
+            checkArgument(fetchSize > 0, "The fetch size must be larger than 0.");
+            this.fetchSize = fetchSize;
+            return this;
+        }
+
+        /**
+         * Sets the batch size of MongoDB find cursor.
+         *
+         * @param cursorBatchSize the max batch size of find cursor.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setCursorBatchSize(int cursorBatchSize) {
+            checkArgument(
+                    cursorBatchSize >= 0,
+                    "The cursor batch size must be larger than or equal to 0.");
+            this.cursorBatchSize = cursorBatchSize;
+            return this;
+        }
+
+        /**
+         * Set this option to true to prevent cursor timeout (10 minutes).

Review Comment:
   Who controls this timeout? Is it a server-side setting that clients may override, or something managed by the mongodb client?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumState.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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;
+
+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.split.MongoScanSourceSplit;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The state class for MongoDB source enumerator, used for storing the split state. This class is
+ * managed and controlled by {@link MongoSplitAssigner}.
+ */
+@PublicEvolving
+public class MongoSourceEnumState {
+
+    /** The Mongo collections remaining. */
+    private final List<String> remainingCollections;
+
+    /**
+     * The paths that are no longer in the enumerator checkpoint, but have been processed before.
+     */
+    private final List<String> alreadyProcessedCollections;

Review Comment:
   Does this keep growing over time?



##########
pom.xml:
##########
@@ -0,0 +1,426 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16.0</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<assertj.version>3.21.0</assertj.version>
+		<archunit.version>0.22.0</archunit.version>
+		<testcontainers.version>1.17.2</testcontainers.version>
+		<mockito.version>3.4.6</mockito.version>
+
+		<japicmp.skip>false</japicmp.skip>
+		<japicmp.referenceVersion>1.15.0</japicmp.referenceVersion>
+
+		<slf4j.version>1.7.36</slf4j.version>
+		<log4j.version>2.17.2</log4j.version>
+
+		<flink.parent.artifactId>flink-connector-mongodb-parent</flink.parent.artifactId>
+	</properties>
+
+	<modules>
+		<module>flink-connector-mongodb</module>
+		<module>flink-sql-connector-mongodb</module>
+		<module>flink-connector-mongodb-e2e-tests</module>
+	</modules>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-force-shading</artifactId>
+			<version>${flink.shaded.version}</version>
+		</dependency>
+
+		<!-- Root dependencies for all projects -->
+
+		<!-- Logging API -->
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- 'javax.annotation' classes like '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.vintage</groupId>
+			<artifactId>junit-vintage-engine</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.assertj</groupId>
+			<artifactId>assertj-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-inline</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-core</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Tests will have log4j as the default logging framework available -->
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-slf4j-impl</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- API bridge between log4j 1 and 2 -->
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-1.2-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-production</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.mongodb</groupId>
+				<artifactId>mongodb-driver-sync</artifactId>
+				<version>${mongodb.version}</version>
+			</dependency>
+
+			<!-- Flink dependencies -->
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-test-utils</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-test-utils-junit</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<!-- Flink ArchUnit -->
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-base</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-test</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-production</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<!-- This manages the 'javax.annotation' annotations (JSR305) -->
+			<dependency>
+				<groupId>com.google.code.findbugs</groupId>
+				<artifactId>jsr305</artifactId>
+				<version>1.3.9</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.slf4j</groupId>
+				<artifactId>slf4j-api</artifactId>
+				<version>${slf4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-slf4j-impl</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-api</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-core</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<!-- API bridge between log4j 1 and 2 -->
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-1.2-api</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>

Review Comment:
   Does the mongodb client work against log4j1? (if not then this shouldnt be required)



##########
pom.xml:
##########
@@ -0,0 +1,426 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>

Review Comment:
   4.7.2 was released in the meantime.



##########
flink-sql-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,95 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-sql-connector-mongodb</artifactId>
+	<name>Flink : Connectors : SQL : MongoDB</name>
+
+	<packaging>jar</packaging>
+
+	<properties>
+		<japicmp.skip>true</japicmp.skip>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet>
+								<includes>
+									<include>org.apache.flink:flink-connector-base</include>
+									<include>org.apache.flink:flink-connector-mongodb</include>
+									<include>org.mongodb:bson</include>
+									<include>org.mongodb:mongodb-driver-sync</include>
+									<include>org.mongodb:mongodb-driver-core</include>
+								</includes>
+							</artifactSet>
+							<filters>
+								<filter>
+									<artifact>org.mongodb:mongodb-driver-core</artifact>
+									<excludes>
+										<exclude>META-INF/native-image/**/**.properties</exclude>

Review Comment:
   Why are these files being excluded?
   Why aren't we just excluding the entire native-image directory?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue() == 1;
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue() == 1L;
+        }
+        if (bsonValue.isString()) {
+            return Boolean.parseBoolean(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to boolean from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static byte convertToTinyInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (byte) 1 : (byte) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (byte) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (byte) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Byte.parseByte(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to tinyint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static short convertToSmallInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (short) 1 : (short) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (short) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (short) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Short.parseShort(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to smallint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static int convertToInt(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().intValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.intValue();
+            } else if (decimal128Value.isNegative()) {
+                return Integer.MIN_VALUE;
+            } else {
+                return Integer.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isDateTime()) {
+            return (int) Instant.ofEpochMilli(bsonValue.asDateTime().getValue()).getEpochSecond();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime();
+        }
+        if (bsonValue.isString()) {
+            return Integer.parseInt(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to integer from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static long convertToLong(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().longValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.longValue();
+            } else if (decimal128Value.isNegative()) {
+                return Long.MIN_VALUE;
+            } else {
+                return Long.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1L : 0L;
+        }
+        if (bsonValue.isDateTime()) {
+            return bsonValue.asDateTime().getValue();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime() * 1000L;
+        }
+        if (bsonValue.isString()) {
+            return Long.parseLong(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to long from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static double convertToDouble(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().doubleValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.doubleValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Double.MAX_VALUE;
+            } else {
+                return Double.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isString()) {
+            return Double.parseDouble(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to double from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static float convertToFloat(BsonValue bsonValue) {
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isDouble()) {
+            return ((Double) bsonValue.asDouble().getValue()).floatValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.floatValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Float.MAX_VALUE;
+            } else {
+                return Float.MAX_VALUE;

Review Comment:
   why isnt this using POSITIVE/NEGATIVE_INFINITY?



##########
flink-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,148 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb</artifactId>
+	<name>Flink : Connectors : MongoDB</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<!-- Core -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-base</artifactId>
+			<version>${flink.version}</version>
+		</dependency>

Review Comment:
   mark as provided (see shade-plugin comment sql connector module).



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+    private static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();

Review Comment:
   use an EnumSet instead



##########
flink-sql-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,95 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-sql-connector-mongodb</artifactId>
+	<name>Flink : Connectors : SQL : MongoDB</name>
+
+	<packaging>jar</packaging>
+
+	<properties>
+		<japicmp.skip>true</japicmp.skip>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet>
+								<includes>
+									<include>org.apache.flink:flink-connector-base</include>
+									<include>org.apache.flink:flink-connector-mongodb</include>
+									<include>org.mongodb:bson</include>
+									<include>org.mongodb:mongodb-driver-sync</include>
+									<include>org.mongodb:mongodb-driver-core</include>
+								</includes>
+							</artifactSet>
+							<filters>
+								<filter>
+									<artifact>org.mongodb:mongodb-driver-core</artifact>
+									<excludes>
+										<exclude>META-INF/native-image/**/**.properties</exclude>
+									</excludes>
+								</filter>
+							</filters>
+							<relocations>
+								<!-- Force relocation of all MongoDB dependencies. -->

Review Comment:
   ```suggestion
   ```
   There isn't really anything "forced" about this.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+    private static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+    static {
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME);
+    }
+
+    /**
+     * Checks that the table does not have a primary key defined on illegal types. In MongoDB the
+     * primary key is used to calculate the MongoDB document id, which is a string of up to 1024
+     * bytes. It cannot have whitespaces. As of now it is calculated by concatenating the fields.
+     * Certain types do not have a good string representation to be used in this scenario. The
+     * illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link
+     * LogicalTypeRoot#RAW} type.
+     */
+    public static void validatePrimaryKey(DataType primaryKeyDataType) {
+        List<DataType> fieldDataTypes = DataType.getFieldDataTypes(primaryKeyDataType);
+        List<LogicalTypeRoot> illegalTypes =
+                fieldDataTypes.stream()
+                        .map(DataType::getLogicalType)
+                        .map(
+                                logicalType -> {
+                                    if (logicalType.is(LogicalTypeRoot.DISTINCT_TYPE)) {
+                                        return ((DistinctType) logicalType)
+                                                .getSourceType()
+                                                .getTypeRoot();
+                                    } else {
+                                        return logicalType.getTypeRoot();
+                                    }
+                                })
+                        .filter(t -> !ALLOWED_PRIMARY_KEY_TYPES.contains(t))
+                        .collect(Collectors.toList());
+        if (!illegalTypes.isEmpty()) {
+            throw new ValidationException(
+                    String.format(
+                            "The table has a primary key on columns of illegal types: %s.",

Review Comment:
   Consider logging the supported data types.
   Is this documented anywhere?



##########
flink-sql-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,95 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-sql-connector-mongodb</artifactId>
+	<name>Flink : Connectors : SQL : MongoDB</name>
+
+	<packaging>jar</packaging>
+
+	<properties>
+		<japicmp.skip>true</japicmp.skip>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet>
+								<includes>
+									<include>org.apache.flink:flink-connector-base</include>

Review Comment:
   Let's not bundle this. It is very problematic w.r.t. API stability (because it may internally rely on internal methods which may change between patch releases).



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+    private static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+    static {
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME);
+    }
+
+    /**
+     * Checks that the table does not have a primary key defined on illegal types. In MongoDB the
+     * primary key is used to calculate the MongoDB document id, which is a string of up to 1024
+     * bytes. It cannot have whitespaces. As of now it is calculated by concatenating the fields.

Review Comment:
   > As of now it is calculated by concatenating the fields.
   
   * Is this a MongoDB or connector detail? (It's always good to link to some source for such statements).
   * 1024 bytes doesn't seem like much when char/varchar are allowed.
   



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.time.Duration;
+
+/**
+ * Base options for the MongoDB connector. Needs to be public so that the {@link
+ * org.apache.flink.table.api.TableDescriptor} can access it.
+ */
+@PublicEvolving
+public class MongoConnectorOptions {
+
+    private MongoConnectorOptions() {}
+
+    public static final ConfigOption<String> URI =
+            ConfigOptions.key("uri")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the connection uri of MongoDB.");
+
+    public static final ConfigOption<String> DATABASE =
+            ConfigOptions.key("database")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the database to read or write of MongoDB.");
+
+    public static final ConfigOption<String> COLLECTION =
+            ConfigOptions.key("collection")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the collection to read or write of MongoDB.");
+
+    public static final ConfigOption<Integer> SCAN_FETCH_SIZE =
+            ConfigOptions.key("scan.fetch-size")
+                    .intType()
+                    .defaultValue(2048)
+                    .withDescription(
+                            "Gives the reader a hint as to the number of documents that should be fetched from the database per round-trip when reading. ");
+
+    public static final ConfigOption<Integer> SCAN_CURSOR_BATCH_SIZE =
+            ConfigOptions.key("scan.cursor.batch-size")
+                    .intType()
+                    .defaultValue(0)
+                    .withDescription(
+                            "Specifies the number of documents to return in each batch of the response from the MongoDB instance. Set to 0 to use server's default.");
+
+    public static final ConfigOption<Boolean> SCAN_CURSOR_NO_TIMEOUT =
+            ConfigOptions.key("scan.cursor.no-timeout")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription(
+                            "The server normally times out idle cursors after an inactivity"
+                                    + " period (10 minutes) to prevent excess memory use. Set this option to true to prevent that."
+                                    + " However, if the application takes longer than 30 minutes to process the current batch of documents,"
+                                    + " the session is marked as expired and closed.");
+
+    public static final ConfigOption<PartitionStrategy> SCAN_PARTITION_STRATEGY =
+            ConfigOptions.key("scan.partition.strategy")
+                    .enumType(PartitionStrategy.class)
+                    .defaultValue(PartitionStrategy.DEFAULT)
+                    .withDescription(
+                            "Specifies the partition strategy. Available strategies are single, sample, split-vector, sharded and default.");
+
+    public static final ConfigOption<MemorySize> SCAN_PARTITION_SIZE =
+            ConfigOptions.key("scan.partition.size")
+                    .memoryType()
+                    .defaultValue(MemorySize.parse("64mb"))
+                    .withDescription("Specifies the partition memory size.");
+
+    public static final ConfigOption<Integer> SCAN_PARTITION_SAMPLES =
+            ConfigOptions.key("scan.partition.samples")
+                    .intType()
+                    .defaultValue(10)
+                    .withDescription(
+                            "Specifies the the samples count per partition. It only takes effect when the partition strategy is sample.");
+
+    public static final ConfigOption<Duration> LOOKUP_RETRY_INTERVAL =
+            ConfigOptions.key("lookup.retry.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofMillis(1000L))
+                    .withDescription(
+                            "Specifies the retry time interval if lookup records from database failed.");
+
+    public static final ConfigOption<Integer> BULK_FLUSH_MAX_ACTIONS =
+            ConfigOptions.key("sink.bulk-flush.max-actions")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "Specifies the maximum number of buffered actions per bulk request.");
+
+    public static final ConfigOption<Duration> BULK_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.bulk-flush.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1))
+                    .withDescription("Specifies the bulk flush interval.");
+
+    public static final ConfigOption<DeliveryGuarantee> DELIVERY_GUARANTEE =
+            ConfigOptions.key("sink.delivery-guarantee")
+                    .enumType(DeliveryGuarantee.class)
+                    .defaultValue(DeliveryGuarantee.AT_LEAST_ONCE)
+                    .withDescription("Optional delivery guarantee when committing.");

Review Comment:
   Document that `exactly-once` is not supported.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.split.MongoScanSourceSplitReader;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The configuration class for MongoDB source. */
+@PublicEvolving
+public class MongoReadOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int fetchSize;
+
+    private final int cursorBatchSize;
+
+    private final boolean noCursorTimeout;
+
+    private final PartitionStrategy partitionStrategy;
+
+    private final MemorySize partitionSize;
+
+    private final int samplesPerPartition;
+
+    private MongoReadOptions(
+            int fetchSize,
+            int cursorBatchSize,
+            boolean noCursorTimeout,
+            PartitionStrategy partitionStrategy,
+            MemorySize partitionSize,
+            int samplesPerPartition) {
+        this.fetchSize = fetchSize;
+        this.cursorBatchSize = cursorBatchSize;
+        this.noCursorTimeout = noCursorTimeout;
+        this.partitionStrategy = partitionStrategy;
+        this.partitionSize = partitionSize;
+        this.samplesPerPartition = samplesPerPartition;
+    }
+
+    public int getFetchSize() {
+        return fetchSize;
+    }
+
+    public int getCursorBatchSize() {
+        return cursorBatchSize;
+    }
+
+    public boolean isNoCursorTimeout() {
+        return noCursorTimeout;
+    }
+
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    public MemorySize getPartitionSize() {
+        return partitionSize;
+    }
+
+    public int getSamplesPerPartition() {
+        return samplesPerPartition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoReadOptions that = (MongoReadOptions) o;
+        return cursorBatchSize == that.cursorBatchSize
+                && noCursorTimeout == that.noCursorTimeout
+                && partitionStrategy == that.partitionStrategy
+                && samplesPerPartition == that.samplesPerPartition
+                && Objects.equals(partitionSize, that.partitionSize);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                cursorBatchSize,
+                noCursorTimeout,
+                partitionStrategy,
+                partitionSize,
+                samplesPerPartition);
+    }
+
+    public static MongoReadOptionsBuilder builder() {
+        return new MongoReadOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoReadOptions}. */
+    @PublicEvolving
+    public static class MongoReadOptionsBuilder {
+        private int fetchSize = SCAN_FETCH_SIZE.defaultValue();
+        private int cursorBatchSize = SCAN_CURSOR_BATCH_SIZE.defaultValue();
+        private boolean noCursorTimeout = SCAN_CURSOR_NO_TIMEOUT.defaultValue();
+        private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue();
+        private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue();
+        private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue();
+
+        /**
+         * Sets the number of documents should be fetched per round-trip when reading.
+         *
+         * @param fetchSize the number of documents should be fetched per round-trip when reading.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setFetchSize(int fetchSize) {
+            checkArgument(fetchSize > 0, "The fetch size must be larger than 0.");
+            this.fetchSize = fetchSize;
+            return this;
+        }
+
+        /**
+         * Sets the batch size of MongoDB find cursor.
+         *
+         * @param cursorBatchSize the max batch size of find cursor.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setCursorBatchSize(int cursorBatchSize) {
+            checkArgument(
+                    cursorBatchSize >= 0,
+                    "The cursor batch size must be larger than or equal to 0.");
+            this.cursorBatchSize = cursorBatchSize;
+            return this;
+        }
+
+        /**
+         * Set this option to true to prevent cursor timeout (10 minutes).
+         *
+         * @param noCursorTimeout Set this option to true to prevent cursor timeout (10 minutes)
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setNoCursorTimeout(boolean noCursorTimeout) {
+            this.noCursorTimeout = noCursorTimeout;
+            return this;
+        }
+
+        /**
+         * Sets the partition strategy. Available partition strategies are single, sample,
+         * split-vector, sharded and default. You can see {@link PartitionStrategy} for detail.
+         *
+         * @param partitionStrategy the strategy of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionStrategy(PartitionStrategy partitionStrategy) {
+            checkNotNull(partitionStrategy, "The partition strategy must not be null.");
+            this.partitionStrategy = partitionStrategy;
+            return this;
+        }
+
+        /**
+         * Sets the partition memory size of MongoDB split. Split a MongoDB collection into multiple
+         * partitions according to the partition memory size. Partitions can be read in parallel by
+         * multiple {@link MongoScanSourceSplitReader} to speed up the overall read time.
+         *
+         * @param partitionSize the memory size of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionSize(MemorySize partitionSize) {
+            checkNotNull(partitionSize, "The partition size must not be null");
+            checkArgument(
+                    partitionSize.getMebiBytes() >= 1,
+                    "The partition size must be larger than or equals to 1mb.");
+            this.partitionSize = partitionSize;
+            return this;
+        }
+
+        /**
+         * Sets the number of samples to take per partition. The total number of samples taken is:
+         * samples per partition * ( count / number of documents per partition).

Review Comment:
   what is `count` in this equation?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+    private static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+    static {
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME);

Review Comment:
   We need some strategy to keep this up-to-date with newly added `LogicalTypeRoots` in Flink.
   Consider explicitly defining an allow/deny set covering the entire enum; that way you can detect newly added roots.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+    private static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+    static {
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME);
+    }
+
+    /**
+     * Checks that the table does not have a primary key defined on illegal types. In MongoDB the
+     * primary key is used to calculate the MongoDB document id, which is a string of up to 1024
+     * bytes. It cannot have whitespaces. As of now it is calculated by concatenating the fields.
+     * Certain types do not have a good string representation to be used in this scenario. The
+     * illegal types are mostly {@link LogicalTypeFamily#COLLECTION} types and {@link
+     * LogicalTypeRoot#RAW} type.
+     */
+    public static void validatePrimaryKey(DataType primaryKeyDataType) {
+        List<DataType> fieldDataTypes = DataType.getFieldDataTypes(primaryKeyDataType);
+        List<LogicalTypeRoot> illegalTypes =
+                fieldDataTypes.stream()
+                        .map(DataType::getLogicalType)
+                        .map(
+                                logicalType -> {
+                                    if (logicalType.is(LogicalTypeRoot.DISTINCT_TYPE)) {

Review Comment:
   This branch would be unnecessary if we would only check the family (because we'd exclude CONSTRUCTED I suppose)



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java:
##########
@@ -0,0 +1,230 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.split.MongoScanSourceSplitReader;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The configuration class for MongoDB source. */
+@PublicEvolving
+public class MongoReadOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int fetchSize;
+
+    private final int cursorBatchSize;
+
+    private final boolean noCursorTimeout;
+
+    private final PartitionStrategy partitionStrategy;
+
+    private final MemorySize partitionSize;
+
+    private final int samplesPerPartition;
+
+    private MongoReadOptions(
+            int fetchSize,
+            int cursorBatchSize,
+            boolean noCursorTimeout,
+            PartitionStrategy partitionStrategy,
+            MemorySize partitionSize,
+            int samplesPerPartition) {
+        this.fetchSize = fetchSize;
+        this.cursorBatchSize = cursorBatchSize;
+        this.noCursorTimeout = noCursorTimeout;
+        this.partitionStrategy = partitionStrategy;
+        this.partitionSize = partitionSize;
+        this.samplesPerPartition = samplesPerPartition;
+    }
+
+    public int getFetchSize() {
+        return fetchSize;
+    }
+
+    public int getCursorBatchSize() {
+        return cursorBatchSize;
+    }
+
+    public boolean isNoCursorTimeout() {
+        return noCursorTimeout;
+    }
+
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    public MemorySize getPartitionSize() {
+        return partitionSize;
+    }
+
+    public int getSamplesPerPartition() {
+        return samplesPerPartition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoReadOptions that = (MongoReadOptions) o;
+        return cursorBatchSize == that.cursorBatchSize
+                && noCursorTimeout == that.noCursorTimeout
+                && partitionStrategy == that.partitionStrategy
+                && samplesPerPartition == that.samplesPerPartition
+                && Objects.equals(partitionSize, that.partitionSize);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                cursorBatchSize,
+                noCursorTimeout,
+                partitionStrategy,
+                partitionSize,
+                samplesPerPartition);
+    }
+
+    public static MongoReadOptionsBuilder builder() {
+        return new MongoReadOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoReadOptions}. */
+    @PublicEvolving
+    public static class MongoReadOptionsBuilder {
+        private int fetchSize = SCAN_FETCH_SIZE.defaultValue();
+        private int cursorBatchSize = SCAN_CURSOR_BATCH_SIZE.defaultValue();
+        private boolean noCursorTimeout = SCAN_CURSOR_NO_TIMEOUT.defaultValue();
+        private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue();
+        private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue();
+        private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue();
+
+        /**
+         * Sets the number of documents should be fetched per round-trip when reading.
+         *
+         * @param fetchSize the number of documents should be fetched per round-trip when reading.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setFetchSize(int fetchSize) {
+            checkArgument(fetchSize > 0, "The fetch size must be larger than 0.");
+            this.fetchSize = fetchSize;
+            return this;
+        }
+
+        /**
+         * Sets the batch size of MongoDB find cursor.
+         *
+         * @param cursorBatchSize the max batch size of find cursor.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setCursorBatchSize(int cursorBatchSize) {
+            checkArgument(
+                    cursorBatchSize >= 0,
+                    "The cursor batch size must be larger than or equal to 0.");
+            this.cursorBatchSize = cursorBatchSize;
+            return this;
+        }
+
+        /**
+         * Set this option to true to prevent cursor timeout (10 minutes).
+         *
+         * @param noCursorTimeout Set this option to true to prevent cursor timeout (10 minutes)
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setNoCursorTimeout(boolean noCursorTimeout) {
+            this.noCursorTimeout = noCursorTimeout;
+            return this;
+        }
+
+        /**
+         * Sets the partition strategy. Available partition strategies are single, sample,
+         * split-vector, sharded and default. You can see {@link PartitionStrategy} for detail.
+         *
+         * @param partitionStrategy the strategy of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionStrategy(PartitionStrategy partitionStrategy) {
+            checkNotNull(partitionStrategy, "The partition strategy must not be null.");
+            this.partitionStrategy = partitionStrategy;
+            return this;
+        }
+
+        /**
+         * Sets the partition memory size of MongoDB split. Split a MongoDB collection into multiple
+         * partitions according to the partition memory size. Partitions can be read in parallel by
+         * multiple {@link MongoScanSourceSplitReader} to speed up the overall read time.
+         *
+         * @param partitionSize the memory size of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionSize(MemorySize partitionSize) {
+            checkNotNull(partitionSize, "The partition size must not be null");
+            checkArgument(
+                    partitionSize.getMebiBytes() >= 1,
+                    "The partition size must be larger than or equals to 1mb.");
+            this.partitionSize = partitionSize;
+            return this;
+        }
+
+        /**
+         * Sets the number of samples to take per partition. The total number of samples taken is:

Review Comment:
   Expand on what this sampling is used for in the javadocs.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/config/MongoConnectionOptions.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.common.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.mongodb.ConnectionString;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The connection configuration class for MongoDB. */
+@PublicEvolving
+public class MongoConnectionOptions implements Serializable {
+
+    private final String uri;
+    private final String database;
+    private final String collection;
+
+    private MongoConnectionOptions(String uri, String database, String collection) {
+        this.uri = checkNotNull(uri);
+        this.database = checkNotNull(database);
+        this.collection = checkNotNull(collection);
+    }
+
+    public String getUri() {
+        return uri;
+    }
+
+    public String getDatabase() {
+        return database;
+    }
+
+    public String getCollection() {
+        return collection;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoConnectionOptions that = (MongoConnectionOptions) o;
+        return Objects.equals(uri, that.uri)
+                && Objects.equals(database, that.database)
+                && Objects.equals(collection, that.collection);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(uri, database, collection);
+    }
+
+    public static MongoConnectionOptionsBuilder builder() {
+        return new MongoConnectionOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoConnectionOptions}. */
+    @PublicEvolving
+    public static class MongoConnectionOptionsBuilder {

Review Comment:
   add a private constructor to enforce a single instantiation path. Please also go over other builds (if more exist)
   



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                retryIntervalMs,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private Integer parallelism;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkArgument(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkArgument(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {

Review Comment:
   ```suggestion
           public MongoWriteOptionsBuilder getMaxRetries(int maxRetryTimes) {
   ```
   propagate naming change



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                retryIntervalMs,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private Integer parallelism;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkArgument(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkArgument(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0,
+                    "The sink max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the retry interval if writing records to database failed.
+         *
+         * @param retryIntervalMs the retry time interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setRetryInterval(long retryIntervalMs) {
+            checkArgument(
+                    retryIntervalMs > 0,
+                    "The retry interval (in milliseconds) must be larger than 0.");
+            this.retryIntervalMs = retryIntervalMs;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#AT_LEAST_ONCE}
+         *
+         * @param deliveryGuarantee which describes the record emission behaviour
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+            checkArgument(
+                    deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+                    "Mongo sink does not support the EXACTLY_ONCE guarantee.");
+            this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+            return this;
+        }
+
+        /**
+         * Sets the parallelism of the Mongo sink operator. By default, the parallelism is
+         * determined by the framework using the same parallelism of the upstream chained operator.
+         */
+        public MongoWriteOptionsBuilder setParallelism(int parallelism) {

Review Comment:
   I don't understand the need for this method.
   
   If a user goes through the DataStream API then they don't need this method (because they can control it in other ways; and we really don't need/want another), while for the Table API we rely on `FactoryUtils#SINK_PARALLELISM` in the factory.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());

Review Comment:
   is there any benefit to this over just creating a `new Object[in.size()];`?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                retryIntervalMs,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private Integer parallelism;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {

Review Comment:
   It would be neat if Flink connectors would use similar terms for similar things.
   
   The JDBC connectors calls this `BatchSize`, and the BulkFlushIntervalMs `BatchIntervalMs`.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializer.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+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.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeList;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeMap;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeList;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeMap;
+
+/** The {@link SimpleVersionedSerializer Serializer} for the enumerator state of Mongo source. */
+@Internal
+public class MongoSourceEnumStateSerializer
+        implements SimpleVersionedSerializer<MongoSourceEnumState> {
+
+    public static final MongoSourceEnumStateSerializer INSTANCE =
+            new MongoSourceEnumStateSerializer();
+
+    private MongoSourceEnumStateSerializer() {
+        // Singleton instance.
+    }
+
+    @Override
+    public int getVersion() {
+        // We use MongoSourceSplitSerializer's version because we use reuse this class.
+        return MongoSourceSplitSerializer.CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(MongoSourceEnumState state) throws IOException {
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                DataOutputStream out = new DataOutputStream(baos)) {
+            serializeList(out, state.getRemainingCollections(), DataOutputStream::writeUTF);
+
+            serializeList(out, state.getAlreadyProcessedCollections(), DataOutputStream::writeUTF);
+
+            serializeList(
+                    out,
+                    state.getRemainingScanSplits(),
+                    MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
+
+            serializeMap(
+                    out,
+                    state.getAssignedScanSplits(),
+                    DataOutputStream::writeUTF,
+                    MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
+
+            out.writeBoolean(state.isInitialized());
+
+            out.flush();
+            return baos.toByteArray();
+        }
+    }
+
+    @Override
+    public MongoSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
+        try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
+                DataInputStream in = new DataInputStream(bais)) {
+            List<String> remainingCollections = deserializeList(in, DataInput::readUTF);
+            List<String> alreadyProcessedCollections = deserializeList(in, DataInput::readUTF);
+            List<MongoScanSourceSplit> remainingScanSplits =
+                    deserializeList(in, i -> deserializeMongoScanSourceSplit(version, i));
+
+            Map<String, MongoScanSourceSplit> assignedScanSplits =
+                    deserializeMap(
+                            in,
+                            DataInput::readUTF,
+                            i -> deserializeMongoScanSourceSplit(version, i));
+
+            boolean initialized = in.readBoolean();
+
+            return new MongoSourceEnumState(
+                    remainingCollections,
+                    alreadyProcessedCollections,
+                    remainingScanSplits,
+                    assignedScanSplits,
+                    initialized);
+        }
+    }
+
+    private MongoScanSourceSplit deserializeMongoScanSourceSplit(int version, DataInputStream in)

Review Comment:
   ```suggestion
       private static MongoScanSourceSplit deserializeMongoScanSourceSplit(int version, DataInputStream in)
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+    private static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+    static {
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME);

Review Comment:
   Would it maybe make more sense to check this via the `LogicalTypeFamily`?



##########
pom.xml:
##########
@@ -0,0 +1,426 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16.0</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<assertj.version>3.21.0</assertj.version>
+		<archunit.version>0.22.0</archunit.version>
+		<testcontainers.version>1.17.2</testcontainers.version>
+		<mockito.version>3.4.6</mockito.version>
+
+		<japicmp.skip>false</japicmp.skip>
+		<japicmp.referenceVersion>1.15.0</japicmp.referenceVersion>
+
+		<slf4j.version>1.7.36</slf4j.version>
+		<log4j.version>2.17.2</log4j.version>
+
+		<flink.parent.artifactId>flink-connector-mongodb-parent</flink.parent.artifactId>
+	</properties>
+
+	<modules>
+		<module>flink-connector-mongodb</module>
+		<module>flink-sql-connector-mongodb</module>
+		<module>flink-connector-mongodb-e2e-tests</module>
+	</modules>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-force-shading</artifactId>
+			<version>${flink.shaded.version}</version>
+		</dependency>
+
+		<!-- Root dependencies for all projects -->
+
+		<!-- Logging API -->
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- 'javax.annotation' classes like '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.vintage</groupId>
+			<artifactId>junit-vintage-engine</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.assertj</groupId>
+			<artifactId>assertj-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-inline</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-core</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Tests will have log4j as the default logging framework available -->
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-slf4j-impl</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- API bridge between log4j 1 and 2 -->
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-1.2-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-production</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.mongodb</groupId>
+				<artifactId>mongodb-driver-sync</artifactId>
+				<version>${mongodb.version}</version>
+			</dependency>
+
+			<!-- Flink dependencies -->
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-test-utils</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-test-utils-junit</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<!-- Flink ArchUnit -->
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-base</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-test</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-production</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<!-- This manages the 'javax.annotation' annotations (JSR305) -->
+			<dependency>
+				<groupId>com.google.code.findbugs</groupId>
+				<artifactId>jsr305</artifactId>
+				<version>1.3.9</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.slf4j</groupId>
+				<artifactId>slf4j-api</artifactId>
+				<version>${slf4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-slf4j-impl</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-api</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-core</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<!-- API bridge between log4j 1 and 2 -->
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-1.2-api</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<!-- For dependency convergence -->
+			<dependency>
+				<groupId>com.fasterxml.jackson</groupId>
+				<artifactId>jackson-bom</artifactId>
+				<type>pom</type>
+				<scope>import</scope>
+				<version>2.13.2.20220328</version>
+			</dependency>
+
+			<!-- For dependency convergence -->
+			<dependency>
+				<groupId>org.junit</groupId>
+				<artifactId>junit-bom</artifactId>
+				<version>${junit5.version}</version>
+				<type>pom</type>
+				<scope>import</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>junit</groupId>
+				<artifactId>junit</artifactId>
+				<version>${junit4.version}</version>
+				<scope>test</scope>
+			</dependency>

Review Comment:
   shouldnt be required; unless there's a good reason for requiring junit4



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializer.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+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.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeList;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeMap;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeList;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeMap;
+
+/** The {@link SimpleVersionedSerializer Serializer} for the enumerator state of Mongo source. */
+@Internal
+public class MongoSourceEnumStateSerializer
+        implements SimpleVersionedSerializer<MongoSourceEnumState> {
+
+    public static final MongoSourceEnumStateSerializer INSTANCE =
+            new MongoSourceEnumStateSerializer();
+
+    private MongoSourceEnumStateSerializer() {
+        // Singleton instance.
+    }
+
+    @Override
+    public int getVersion() {
+        // We use MongoSourceSplitSerializer's version because we use reuse this class.
+        return MongoSourceSplitSerializer.CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(MongoSourceEnumState state) throws IOException {
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                DataOutputStream out = new DataOutputStream(baos)) {
+            serializeList(out, state.getRemainingCollections(), DataOutputStream::writeUTF);
+
+            serializeList(out, state.getAlreadyProcessedCollections(), DataOutputStream::writeUTF);
+
+            serializeList(
+                    out,
+                    state.getRemainingScanSplits(),
+                    MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
+
+            serializeMap(
+                    out,
+                    state.getAssignedScanSplits(),
+                    DataOutputStream::writeUTF,
+                    MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
+
+            out.writeBoolean(state.isInitialized());
+
+            out.flush();
+            return baos.toByteArray();
+        }
+    }
+
+    @Override
+    public MongoSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
+        try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
+                DataInputStream in = new DataInputStream(bais)) {
+            List<String> remainingCollections = deserializeList(in, DataInput::readUTF);
+            List<String> alreadyProcessedCollections = deserializeList(in, DataInput::readUTF);
+            List<MongoScanSourceSplit> remainingScanSplits =
+                    deserializeList(in, i -> deserializeMongoScanSourceSplit(version, i));
+
+            Map<String, MongoScanSourceSplit> assignedScanSplits =
+                    deserializeMap(
+                            in,
+                            DataInput::readUTF,
+                            i -> deserializeMongoScanSourceSplit(version, i));
+
+            boolean initialized = in.readBoolean();
+
+            return new MongoSourceEnumState(
+                    remainingCollections,
+                    alreadyProcessedCollections,
+                    remainingScanSplits,
+                    assignedScanSplits,
+                    initialized);
+        }
+    }
+
+    private MongoScanSourceSplit deserializeMongoScanSourceSplit(int version, DataInputStream in)
+            throws IOException {
+        return (MongoScanSourceSplit)

Review Comment:
   Why this cast required?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue() == 1;
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue() == 1L;
+        }
+        if (bsonValue.isString()) {
+            return Boolean.parseBoolean(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to boolean from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static byte convertToTinyInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (byte) 1 : (byte) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (byte) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (byte) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Byte.parseByte(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to tinyint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static short convertToSmallInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (short) 1 : (short) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (short) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (short) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Short.parseShort(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to smallint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static int convertToInt(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().intValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.intValue();
+            } else if (decimal128Value.isNegative()) {
+                return Integer.MIN_VALUE;
+            } else {
+                return Integer.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isDateTime()) {
+            return (int) Instant.ofEpochMilli(bsonValue.asDateTime().getValue()).getEpochSecond();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime();
+        }
+        if (bsonValue.isString()) {
+            return Integer.parseInt(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to integer from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static long convertToLong(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().longValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.longValue();
+            } else if (decimal128Value.isNegative()) {
+                return Long.MIN_VALUE;
+            } else {
+                return Long.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1L : 0L;
+        }
+        if (bsonValue.isDateTime()) {
+            return bsonValue.asDateTime().getValue();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime() * 1000L;
+        }
+        if (bsonValue.isString()) {
+            return Long.parseLong(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to long from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static double convertToDouble(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().doubleValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.doubleValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Double.MAX_VALUE;
+            } else {
+                return Double.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isString()) {
+            return Double.parseDouble(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to double from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static float convertToFloat(BsonValue bsonValue) {
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {

Review Comment:
   I'm very sceptical about all this conversion logic.
   
   AFAICT what you're doing here are mostly workarounds for errors by the user, trying to reconcile the actual type with the schema that the user provided.
   
   In short: Don't. Have it fail so users fix their stuff to prevent surprises down the line, and have them rely on the table api casting to convert types if needed.
   Exceptions can be made for types that don't have a good 1:1 mapping, like javascript code (seriously why is that a special type :facepalm:)



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_FETCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SAMPLES;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_STRATEGY;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The configuration class for MongoDB source. */
+@PublicEvolving
+public class MongoReadOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int fetchSize;
+
+    private final int cursorBatchSize;
+
+    private final boolean noCursorTimeout;
+
+    private final PartitionStrategy partitionStrategy;
+
+    private final MemorySize partitionSize;
+
+    private final int samplesPerPartition;
+
+    private MongoReadOptions(
+            int fetchSize,
+            int cursorBatchSize,
+            boolean noCursorTimeout,
+            PartitionStrategy partitionStrategy,
+            MemorySize partitionSize,
+            int samplesPerPartition) {
+        this.fetchSize = fetchSize;
+        this.cursorBatchSize = cursorBatchSize;
+        this.noCursorTimeout = noCursorTimeout;
+        this.partitionStrategy = partitionStrategy;
+        this.partitionSize = partitionSize;
+        this.samplesPerPartition = samplesPerPartition;
+    }
+
+    public int getFetchSize() {
+        return fetchSize;
+    }
+
+    public int getCursorBatchSize() {
+        return cursorBatchSize;
+    }
+
+    public boolean isNoCursorTimeout() {
+        return noCursorTimeout;
+    }
+
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    public MemorySize getPartitionSize() {
+        return partitionSize;
+    }
+
+    public int getSamplesPerPartition() {
+        return samplesPerPartition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoReadOptions that = (MongoReadOptions) o;
+        return cursorBatchSize == that.cursorBatchSize
+                && noCursorTimeout == that.noCursorTimeout
+                && partitionStrategy == that.partitionStrategy
+                && samplesPerPartition == that.samplesPerPartition
+                && Objects.equals(partitionSize, that.partitionSize);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                cursorBatchSize,
+                noCursorTimeout,
+                partitionStrategy,
+                partitionSize,
+                samplesPerPartition);
+    }
+
+    public static MongoReadOptionsBuilder builder() {
+        return new MongoReadOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoReadOptions}. */
+    public static class MongoReadOptionsBuilder {
+        private int fetchSize = SCAN_FETCH_SIZE.defaultValue();
+        private int cursorBatchSize = SCAN_CURSOR_BATCH_SIZE.defaultValue();
+        private boolean noCursorTimeout = SCAN_CURSOR_NO_TIMEOUT.defaultValue();
+        private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue();
+        private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue();
+        private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue();
+
+        /**
+         * Sets the number of documents should be fetched per round-trip when reading.
+         *
+         * @param fetchSize the number of documents should be fetched per round-trip when reading.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setFetchSize(int fetchSize) {
+            checkArgument(fetchSize > 0, "The fetch size must be larger than 0.");
+            this.fetchSize = fetchSize;
+            return this;
+        }
+
+        /**
+         * Sets the batch size of MongoDB find cursor.
+         *
+         * @param cursorBatchSize the max batch size of find cursor.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setCursorBatchSize(int cursorBatchSize) {
+            checkArgument(
+                    cursorBatchSize >= 0,
+                    "The cursor batch size must be larger than or equal to 0.");
+            this.cursorBatchSize = cursorBatchSize;
+            return this;
+        }
+
+        /**
+         * Set this option to true to prevent cursor timeout (10 minutes).
+         *
+         * @param noCursorTimeout Set this option to true to prevent cursor timeout (10 minutes)
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setNoCursorTimeout(boolean noCursorTimeout) {
+            this.noCursorTimeout = noCursorTimeout;
+            return this;
+        }
+
+        /**
+         * Sets the partition strategy.
+         *
+         * @param partitionStrategy the strategy of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionStrategy(PartitionStrategy partitionStrategy) {
+            checkNotNull(partitionStrategy, "The partition strategy must not be null.");
+            this.partitionStrategy = partitionStrategy;
+            return this;
+        }
+
+        /**
+         * Sets the partition size of MongoDB split.
+         *
+         * @param partitionSize the memory size of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionSize(MemorySize partitionSize) {
+            checkNotNull(partitionSize, "The partition size must not be null");
+            checkArgument(
+                    partitionSize.getMebiBytes() >= 1,
+                    "The partition size must be larger than or equals to 1mb.");

Review Comment:
   let's document this here with a comment.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/architecture/ProductionCodeArchitectureTest.java:
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.architecture;
+
+import org.apache.flink.architecture.common.ImportOptions;
+
+import com.tngtech.archunit.core.importer.ImportOption;
+import com.tngtech.archunit.junit.AnalyzeClasses;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.junit.ArchTests;
+
+/** Architecture tests for production code. */
+@AnalyzeClasses(
+        packages = "org.apache.flink.connector",

Review Comment:
   ```suggestion
           packages = "org.apache.flink.connector.mongodb",
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017723597


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                retryIntervalMs,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private Integer parallelism;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkArgument(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkArgument(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0,
+                    "The sink max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the retry interval if writing records to database failed.
+         *
+         * @param retryIntervalMs the retry time interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setRetryInterval(long retryIntervalMs) {
+            checkArgument(
+                    retryIntervalMs > 0,
+                    "The retry interval (in milliseconds) must be larger than 0.");
+            this.retryIntervalMs = retryIntervalMs;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#AT_LEAST_ONCE}
+         *
+         * @param deliveryGuarantee which describes the record emission behaviour
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+            checkArgument(
+                    deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+                    "Mongo sink does not support the EXACTLY_ONCE guarantee.");
+            this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+            return this;
+        }
+
+        /**
+         * Sets the parallelism of the Mongo sink operator. By default, the parallelism is
+         * determined by the framework using the same parallelism of the upstream chained operator.
+         */
+        public MongoWriteOptionsBuilder setParallelism(int parallelism) {

Review Comment:
   Got it.  
   I'll pass the parallelism explicitly to the TableSink constructor instead.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017697303


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/config/MongoConnectionOptions.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.common.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.mongodb.ConnectionString;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The connection configuration class for MongoDB. */
+@PublicEvolving
+public class MongoConnectionOptions implements Serializable {
+
+    private final String uri;
+    private final String database;
+    private final String collection;
+
+    private MongoConnectionOptions(String uri, String database, String collection) {
+        this.uri = checkNotNull(uri);
+        this.database = checkNotNull(database);
+        this.collection = checkNotNull(collection);
+    }
+
+    public String getUri() {
+        return uri;
+    }
+
+    public String getDatabase() {
+        return database;
+    }
+
+    public String getCollection() {
+        return collection;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoConnectionOptions that = (MongoConnectionOptions) o;
+        return Objects.equals(uri, that.uri)
+                && Objects.equals(database, that.database)
+                && Objects.equals(collection, that.collection);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(uri, database, collection);
+    }
+
+    public static MongoConnectionOptionsBuilder builder() {
+        return new MongoConnectionOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoConnectionOptions}. */
+    @PublicEvolving
+    public static class MongoConnectionOptionsBuilder {
+        private String uri;
+        private String database;
+        private String collection;
+
+        /**
+         * Sets the connection string of MongoDB.
+         *
+         * @param uri connection string of MongoDB
+         * @return this builder
+         */
+        public MongoConnectionOptionsBuilder setUri(String uri) {
+            this.uri = new ConnectionString(uri).getConnectionString();
+            return this;

Review Comment:
   I glanced over the code but just to double-check: it doesn't try to resolve the host, correct?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025476043


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";

Review Comment:
   Changed to private



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017034256


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+    private static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+    static {
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME);
+    }
+
+    /**
+     * Checks that the table does not have a primary key defined on illegal types. In MongoDB the
+     * primary key is used to calculate the MongoDB document id, which is a string of up to 1024
+     * bytes. It cannot have whitespaces. As of now it is calculated by concatenating the fields.

Review Comment:
   Before MongoDB 4.2, the total size of an index entry, which can include structural overhead depending on the BSON type, must be less than 1024 bytes. Starting in version 4.2, MongoDB removes the index key limit.
   Please refers to https://www.mongodb.com/docs/manual/reference/limits/#mongodb-limit-Index-Key-Limit for detail.
   
   Actually any bson types (exclude array) can be used for _id, the comments are a bit inaccurate, I will make some adjustments.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010743536


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_FETCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SAMPLES;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_STRATEGY;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The configuration class for MongoDB source. */
+@PublicEvolving
+public class MongoReadOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int fetchSize;
+
+    private final int cursorBatchSize;
+
+    private final boolean noCursorTimeout;
+
+    private final PartitionStrategy partitionStrategy;
+
+    private final MemorySize partitionSize;
+
+    private final int samplesPerPartition;
+
+    private MongoReadOptions(
+            int fetchSize,
+            int cursorBatchSize,
+            boolean noCursorTimeout,
+            PartitionStrategy partitionStrategy,
+            MemorySize partitionSize,
+            int samplesPerPartition) {
+        this.fetchSize = fetchSize;
+        this.cursorBatchSize = cursorBatchSize;
+        this.noCursorTimeout = noCursorTimeout;
+        this.partitionStrategy = partitionStrategy;
+        this.partitionSize = partitionSize;
+        this.samplesPerPartition = samplesPerPartition;
+    }
+
+    public int getFetchSize() {
+        return fetchSize;
+    }
+
+    public int getCursorBatchSize() {
+        return cursorBatchSize;
+    }
+
+    public boolean isNoCursorTimeout() {
+        return noCursorTimeout;
+    }
+
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    public MemorySize getPartitionSize() {
+        return partitionSize;
+    }
+
+    public int getSamplesPerPartition() {
+        return samplesPerPartition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoReadOptions that = (MongoReadOptions) o;
+        return cursorBatchSize == that.cursorBatchSize
+                && noCursorTimeout == that.noCursorTimeout
+                && partitionStrategy == that.partitionStrategy
+                && samplesPerPartition == that.samplesPerPartition
+                && Objects.equals(partitionSize, that.partitionSize);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                cursorBatchSize,
+                noCursorTimeout,
+                partitionStrategy,
+                partitionSize,
+                samplesPerPartition);
+    }
+
+    public static MongoReadOptionsBuilder builder() {
+        return new MongoReadOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoReadOptions}. */
+    public static class MongoReadOptionsBuilder {
+        private int fetchSize = SCAN_FETCH_SIZE.defaultValue();
+        private int cursorBatchSize = SCAN_CURSOR_BATCH_SIZE.defaultValue();
+        private boolean noCursorTimeout = SCAN_CURSOR_NO_TIMEOUT.defaultValue();
+        private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue();
+        private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue();
+        private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue();
+
+        /**
+         * Sets the number of documents should be fetched per round-trip when reading.
+         *
+         * @param fetchSize the number of documents should be fetched per round-trip when reading.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setFetchSize(int fetchSize) {
+            checkArgument(fetchSize > 0, "The fetch size must be larger than 0.");
+            this.fetchSize = fetchSize;
+            return this;
+        }
+
+        /**
+         * Sets the batch size of MongoDB find cursor.
+         *
+         * @param cursorBatchSize the max batch size of find cursor.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setCursorBatchSize(int cursorBatchSize) {
+            checkArgument(
+                    cursorBatchSize >= 0,
+                    "The cursor batch size must be larger than or equal to 0.");
+            this.cursorBatchSize = cursorBatchSize;
+            return this;
+        }
+
+        /**
+         * Set this option to true to prevent cursor timeout (10 minutes).
+         *
+         * @param noCursorTimeout Set this option to true to prevent cursor timeout (10 minutes)
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setNoCursorTimeout(boolean noCursorTimeout) {
+            this.noCursorTimeout = noCursorTimeout;
+            return this;
+        }
+
+        /**
+         * Sets the partition strategy.
+         *
+         * @param partitionStrategy the strategy of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionStrategy(PartitionStrategy partitionStrategy) {
+            checkNotNull(partitionStrategy, "The partition strategy must not be null.");
+            this.partitionStrategy = partitionStrategy;
+            return this;
+        }
+
+        /**
+         * Sets the partition size of MongoDB split.
+         *
+         * @param partitionSize the memory size of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionSize(MemorySize partitionSize) {
+            checkNotNull(partitionSize, "The partition size must not be null");
+            checkArgument(
+                    partitionSize.getMebiBytes() >= 1,
+                    "The partition size must be larger than or equals to 1mb.");
+            this.partitionSize = partitionSize;
+            return this;
+        }
+
+        /**
+         * Sets the partition size of MongoDB split.

Review Comment:
   How about change it to the following?
   ```
   Sets the partition memory size of MongoDB split. Split a MongoDB collection into multiple
   partitions according to the partition memory size. We can read these partitions in parallel
   to improve the reading speed.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011146094


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SINK_MAX_RETRIES;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The configured class for Mongo sink. */
+@PublicEvolving
+public class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final @Nullable Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private @Nullable Integer parallelism;
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkState(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkState(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0, "The max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#NONE}
+         *
+         * @param deliveryGuarantee which describes the record emission behaviour
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+            checkState(
+                    deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+                    "Mongo sink does not support the EXACTLY_ONCE guarantee.");
+            this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+            return this;
+        }
+
+        /**
+         * Sets the write parallelism.

Review Comment:
   Sets the parallelism of the Mongo sink operator. 
   By default, the parallelism is determined by the framework using the same parallelism of the upstream chained operator. 
   User-defined parallel parallelism ​​can be passed to SinkV2Provider through this configuration.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011195989


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumerator.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.mongodb.source.MongoSource;
+import org.apache.flink.connector.mongodb.source.enumerator.assigner.MongoSplitAssigner;
+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.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.TreeSet;
+
+/** The enumerator class for {@link MongoSource}. */
+@Internal
+public class MongoSourceEnumerator
+        implements SplitEnumerator<MongoSourceSplit, MongoSourceEnumState> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSourceEnumerator.class);
+
+    private final Boundedness boundedness;
+    private final SplitEnumeratorContext<MongoSourceSplit> context;
+    private final MongoSplitAssigner splitAssigner;
+    private final TreeSet<Integer> readersAwaitingSplit;
+
+    public MongoSourceEnumerator(
+            Boundedness boundedness,
+            SplitEnumeratorContext<MongoSourceSplit> context,
+            MongoSplitAssigner splitAssigner) {
+        this.boundedness = boundedness;
+        this.context = context;
+        this.splitAssigner = splitAssigner;
+        this.readersAwaitingSplit = new TreeSet<>();
+    }
+
+    @Override
+    public void start() {
+        splitAssigner.open();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        if (!context.registeredReaders().containsKey(subtaskId)) {
+            // reader failed between sending the request and now. skip this request.
+            return;
+        }
+
+        readersAwaitingSplit.add(subtaskId);
+        assignSplits();
+    }
+
+    @Override
+    public void addSplitsBack(List<MongoSourceSplit> splits, int subtaskId) {
+        LOG.debug("Mongo Source Enumerator adds splits back: {}", splits);
+        splitAssigner.addSplitsBack(splits);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        LOG.debug("Adding reader {} to MongoSourceEnumerator.", subtaskId);

Review Comment:
   Mongo source is purely lazy-pull-based, nothing to do upon registration.
   Splits are assigned at handleSplitRequest and can be stolen by readers that have already completed a split reading.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1095898919


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/RowDataToBsonConverters.java:
##########
@@ -0,0 +1,317 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.ArrayData;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonNull;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.json.JsonParseException;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+
+/** Tool class used to convert from {@link RowData} to {@link BsonValue}. */
+@Internal
+public class RowDataToBsonConverters {
+
+    // --------------------------------------------------------------------------------
+    // Runtime Converters
+    // --------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts objects of Flink Table & SQL internal data structures to
+     * corresponding {@link BsonValue} data structures.
+     */
+    @FunctionalInterface
+    public interface RowDataToBsonConverter extends Serializable {
+        BsonValue convert(Object value);
+    }

Review Comment:
   ~~This interface is for the conversion of specific sql types later, and the parameter may not be clearly defined as `SqlData`. 
   For example, `DecimalData` and `TimestampData` do not have a specific parent class like `SqlData`.~~
   
   `BsonToRowDataConverter.createConverter(LogicalType)` has been modified as `BsonToRowDataConverter.createConverter(RowType)`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1096059827


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.

Review Comment:
   Yes, it is a random sampling method, not evenly divided. Increasing the number of samples means increasing the sampling rate to try to make the partitions more even.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097121821


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.time.Duration;
+
+/**
+ * Base options for the MongoDB connector. Needs to be public so that the {@link
+ * org.apache.flink.table.api.TableDescriptor} can access it.
+ */
+@PublicEvolving
+public class MongoConnectorOptions {
+
+    private MongoConnectorOptions() {}
+
+    public static final ConfigOption<String> URI =
+            ConfigOptions.key("uri")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the connection uri of MongoDB.");
+
+    public static final ConfigOption<String> DATABASE =
+            ConfigOptions.key("database")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the database to read or write of MongoDB.");
+
+    public static final ConfigOption<String> COLLECTION =
+            ConfigOptions.key("collection")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the collection to read or write of MongoDB.");
+
+    public static final ConfigOption<Integer> SCAN_FETCH_SIZE =
+            ConfigOptions.key("scan.fetch-size")
+                    .intType()
+                    .defaultValue(2048)
+                    .withDescription(
+                            "Gives the reader a hint as to the number of documents that should be fetched from the database per round-trip when reading. ");
+
+    public static final ConfigOption<Integer> SCAN_CURSOR_BATCH_SIZE =

Review Comment:
   Thanks @zentol to catch this.
   The original intention of introducing `cusro.batch-size` is that this configuration is similar to 'select from limit'  to limit the number of documents returned to reduce memory overhead. 
   
   I checked some MongoDB documents and found that `cusor.batch-size` will only return up to 16mb of data.
   This is meaningless for protecting the memory size, so I deleted this configuration item.
   
   
   https://www.mongodb.com/docs/manual/tutorial/iterate-a-cursor/#cursor-batches
   
   > The MongoDB server returns the query results in batches. The amount of data in the batch will not exceed the [maximum BSON document size](https://www.mongodb.com/docs/manual/reference/limits/#std-label-limit-bson-document-size). To override the default size of the batch, see [batchSize()](https://www.mongodb.com/docs/manual/reference/method/cursor.batchSize/#mongodb-method-cursor.batchSize) and [limit().](https://www.mongodb.com/docs/manual/reference/method/cursor.limit/#mongodb-method-cursor.limit)
   
   https://www.mongodb.com/community/forums/t/mis-understanding-batchsize/169713
   > find() and aggregate() operations have an initial batch size of 101 documents by default. Subsequent [getMore 10](https://www.mongodb.com/docs/manual/reference/command/getMore/#mongodb-dbcommand-dbcmd.getMore) operations issued against the resulting cursor have no default batch size, so they are limited only by the 16 megabyte message size.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097361445


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriterITCase.java:
##########
@@ -0,0 +1,424 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.connector.mongodb.testutils.MongoTestUtil;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.DeleteOneModel;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.UpdateOneModel;
+import com.mongodb.client.model.UpdateOptions;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+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.util.Optional;
+import java.util.OptionalLong;
+
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreNotWritten;
+import static org.apache.flink.connector.mongodb.testutils.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link MongoWriter}. */
+@Testcontainers
+public class MongoWriterITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriterITCase.class);
+
+    private static final String TEST_DATABASE = "test_writer";
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    private static MongoClient mongoClient;
+    private static MetricListener metricListener;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void setUp() {
+        metricListener = new MetricListener();
+    }
+
+    @Test
+    void testWriteOnBulkFlush() throws Exception {
+        final String collection = "test-bulk-flush-without-checkpoint";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 5;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+
+            // Ignore flush on checkpoint
+            writer.flush(false);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3, 4);
+
+            // Trigger flush
+            writer.write(buildMessage(5), null);
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5);
+
+            writer.write(buildMessage(6), null);
+            assertThatIdsAreNotWritten(collectionOf(collection), 6);
+
+            // Force flush
+            writer.doBulkWrite();
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5, 6);
+        }
+    }
+
+    @Test
+    void testWriteOnBatchIntervalFlush() throws Exception {
+        final String collection = "test-bulk-flush-with-interval";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = -1;
+        final int batchIntervalMs = 1000;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+            writer.doBulkWrite();
+        }
+
+        assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4);
+    }
+
+    @Test
+    void testWriteOnCheckpoint() throws Exception {
+        final String collection = "test-bulk-flush-with-checkpoint";
+        final boolean flushOnCheckpoint = true;
+        final int batchSize = -1;
+        final int batchIntervalMs = -1;
+
+        // Enable flush on checkpoint
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3);
+
+            // Trigger flush
+            writer.flush(false);
+
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3);
+        }
+    }
+
+    @Test
+    void testIncrementRecordsSendMetric() throws Exception {
+        final String collection = "test-inc-records-send";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Counter> recordsSend =
+                    metricListener.getCounter(MetricNames.NUM_RECORDS_SEND);
+            writer.write(buildMessage(1), null);
+            // Update existing index
+            writer.write(buildMessage(2, "u"), null);
+            // Delete index
+            writer.write(buildMessage(3, "d"), null);
+
+            writer.doBulkWrite();
+
+            assertThat(recordsSend.isPresent()).isTrue();
+            assertThat(recordsSend.get().getCount()).isEqualTo(3L);
+        }
+    }
+
+    @Test
+    void testCurrentSendTime() throws Exception {
+        final String collection = "test-current-send-time";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Gauge<Long>> currentSendTime =
+                    metricListener.getGauge("currentSendTime");
+
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+
+            writer.doBulkWrite();
+
+            assertThat(currentSendTime.isPresent()).isTrue();
+            assertThat(currentSendTime.get().getValue()).isGreaterThan(0L);

Review Comment:
   Thanks @zentol  for reminding me of this problem. 
   Retry mechanism has been added into this test case to make it more stable.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1097675063


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSampleSplitter.class);
+
+    public static final MongoSampleSplitter INSTANCE = new MongoSampleSplitter();
+
+    private MongoSampleSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+
+        long count = splitContext.getCount();
+        long partitionSizeInBytes = readOptions.getPartitionSize().getBytes();
+        int samplesPerPartition = readOptions.getSamplesPerPartition();
+
+        long avgObjSizeInBytes = splitContext.getAvgObjSize();
+        if (avgObjSizeInBytes == 0L) {
+            LOG.info(
+                    "{} seems to be an empty collection, Returning a single partition.", namespace);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        long numDocumentsPerPartition = partitionSizeInBytes / avgObjSizeInBytes;
+        if (numDocumentsPerPartition >= count) {
+            LOG.info(
+                    "Fewer documents ({}) than the number of documents per partition ({}), Returning a single partition.",
+                    count,
+                    numDocumentsPerPartition);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        int numberOfSamples =
+                (int) Math.ceil((samplesPerPartition * count * 1.0d) / numDocumentsPerPartition);
+
+        List<BsonDocument> samples =
+                splitContext
+                        .getMongoCollection()
+                        .aggregate(
+                                Arrays.asList(
+                                        Aggregates.sample(numberOfSamples),
+                                        Aggregates.project(Projections.include(ID_FIELD)),
+                                        Aggregates.sort(Sorts.ascending(ID_FIELD))))
+                        .allowDiskUse(true)
+                        .into(new ArrayList<>());

Review Comment:
   > I have refactored the splitter using MIN to replace the first sample and MAX to replace the last sample.
   
   I'm not sure if this way is better than two fewer samples, but it's easier for coding. 
   If this way is not very good, I will adjust it again.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1098406960


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter {

Review Comment:
   That's more of a functional IT test though; answering "does it read all data?". A unit test for the exact set of created splits would be good to have to make sure we dont create empty splits, or have one split containing 99% of the data.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "Jiabao-Sun (via GitHub)" <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1096941489


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import org.bson.BsonDocument;
+import org.bson.BsonType;
+import org.bson.BsonValue;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.DEFAULT_JSON_WRITER_SETTINGS;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ENCODE_VALUE_FIELD;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {

Review Comment:
   Redefined as below.
   ```java
       @FunctionalInterface
       public interface BsonToRowDataConverter extends Serializable {
           RowData convert(BsonDocument bsonDocument);
       }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010744798


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSourceBuilder.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema;
+
+import org.bson.BsonDocument;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.flink.util.CollectionUtil.isNullOrEmpty;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The builder class for {@link MongoSource} to make it easier for the users to construct a {@link
+ * MongoSource}.
+ *
+ * @param <OUT> The output type of the source.
+ */
+@PublicEvolving
+public class MongoSourceBuilder<OUT> {
+
+    private final MongoConnectionOptions.MongoConnectionOptionsBuilder connectionOptionsBuilder;
+    private final MongoReadOptions.MongoReadOptionsBuilder readOptionsBuilder;
+
+    private List<String> projectedFields;
+    private int limit = -1;
+    private MongoDeserializationSchema<OUT> deserializationSchema;
+
+    MongoSourceBuilder() {
+        this.connectionOptionsBuilder = MongoConnectionOptions.builder();
+        this.readOptionsBuilder = MongoReadOptions.builder();
+    }
+
+    /**
+     * Sets the connection string of MongoDB.
+     *
+     * @param uri connection string of MongoDB
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setUri(String uri) {
+        connectionOptionsBuilder.setUri(uri);
+        return this;
+    }
+
+    /**
+     * Sets the database to sink of MongoDB.
+     *
+     * @param database the database to sink of MongoDB.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setDatabase(String database) {
+        connectionOptionsBuilder.setDatabase(database);
+        return this;
+    }
+
+    /**
+     * Sets the collection to sink of MongoDB.
+     *
+     * @param collection the collection to sink of MongoDB.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setCollection(String collection) {
+        connectionOptionsBuilder.setCollection(collection);
+        return this;
+    }
+
+    /**
+     * Sets the number of documents should be fetched per round-trip when reading.
+     *
+     * @param fetchSize the number of documents should be fetched per round-trip when reading.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setFetchSize(int fetchSize) {
+        readOptionsBuilder.setFetchSize(fetchSize);
+        return this;
+    }
+
+    /**
+     * Sets the batch size of MongoDB find cursor.
+     *
+     * @param cursorBatchSize the max batch size of find cursor.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setCursorBatchSize(int cursorBatchSize) {
+        readOptionsBuilder.setCursorBatchSize(cursorBatchSize);
+        return this;
+    }
+
+    /**
+     * Set this option to true to prevent cursor timeout (defaults to 10 minutes).
+     *
+     * @param noCursorTimeout Set this option to true to prevent cursor timeout.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setNoCursorTimeout(boolean noCursorTimeout) {
+        readOptionsBuilder.setNoCursorTimeout(noCursorTimeout);
+        return this;
+    }
+
+    /**
+     * Sets the partition strategy.
+     *
+     * @param partitionStrategy the strategy of a partition.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setPartitionStrategy(PartitionStrategy partitionStrategy) {
+        readOptionsBuilder.setPartitionStrategy(partitionStrategy);
+        return this;
+    }
+
+    /**
+     * Sets the partition size of MongoDB split.
+     *
+     * @param partitionSize the memory size of a partition.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setPartitionSize(MemorySize partitionSize) {
+        readOptionsBuilder.setPartitionSize(partitionSize);
+        return this;
+    }
+
+    /**
+     * Sets the samples size per partition only effective for sample partition strategy.
+     *
+     * @param samplesPerPartition the samples size per partition
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setSamplesPerPartition(int samplesPerPartition) {
+        readOptionsBuilder.setSamplesPerPartition(samplesPerPartition);
+        return this;
+    }
+
+    /**
+     * Sets the limit of documents to read.

Review Comment:
   How about change it to the following?
   ```
   Sets the limit of documents to read. If limit is not set or set to -1, we will not push the limit down to MongoDB.
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010520700


##########
pom.xml:
##########
@@ -0,0 +1,457 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16-SNAPSHOT</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<hamcrest.version>1.3</hamcrest.version>

Review Comment:
   This shouldn't be required because all assertions should go through assertj.



##########
pom.xml:
##########
@@ -0,0 +1,457 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0.0-SNAPSHOT</version>

Review Comment:
   ```suggestion
   	<version>1.0-SNAPSHOT</version>
   ```



##########
flink-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,165 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0.0-SNAPSHOT</version>
+		<relativePath>../pom.xml</relativePath>

Review Comment:
   ```suggestion
   		<version>1.0-SNAPSHOT</version>
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoSerdeUtils.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.FunctionWithException;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** A util class with some helper method for serde in the MongoDB source. */
+@Internal
+public class MongoSerdeUtils {

Review Comment:
   This class is lacking tests.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/MongoSink.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+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.sink.writer.MongoWriter;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+
+import com.mongodb.client.model.WriteModel;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * MongoDB sink that requests multiple {@link WriteModel bulkRequests} against a cluster for each
+ * incoming element. The following example shows how to create a MongoSink receiving records of

Review Comment:
   Sounds strange that we do multiple requests for each element; please expand on that.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/MongoSink.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+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.sink.writer.MongoWriter;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+
+import com.mongodb.client.model.WriteModel;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * MongoDB sink that requests multiple {@link WriteModel bulkRequests} against a cluster for each
+ * incoming element. The following example shows how to create a MongoSink receiving records of
+ * {@code Document} type.
+ *
+ * <pre>{@code
+ * MongoSink<Document> sink = MongoSink.<Document>builder()
+ *     .setUri("mongodb://user:password@127.0.0.1:27017")
+ *     .setDatabase("db")
+ *     .setCollection("coll")
+ *     .setBulkFlushMaxActions(5)
+ *     .setSerializationSchema(
+ *         (doc, context) -> new InsertOneModel<>(doc.toBsonDocument()))
+ *     .build();
+ * }</pre>
+ *
+ * @param <IN> Type of the elements handled by this sink
+ */
+@PublicEvolving
+public class MongoSink<IN> implements Sink<IN> {
+
+    private static final long serialVersionUID = 1L;
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+
+    MongoSink(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);

Review Comment:
   You should use the ClosureCleaner here to ensure the schema is actually serializable.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SINK_MAX_RETRIES;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The configured class for Mongo sink. */

Review Comment:
   This sentence doesn't make any sense.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/context/MongoSinkContext.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.context;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+
+/** This context provides information for {@link MongoSerializationSchema}. */
+@PublicEvolving
+public interface MongoSinkContext {

Review Comment:
   Consider extending `Sink.InitContext`



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SINK_MAX_RETRIES;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The configured class for Mongo sink. */
+@PublicEvolving
+public class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final @Nullable Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    public static class MongoWriteOptionsBuilder {

Review Comment:
   add a private constructor to enforce a single instantiation path.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSource.java:
##########
@@ -0,0 +1,186 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.Source;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+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.synchronization.FutureCompletingBlockingQueue;
+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.enumerator.MongoSourceEnumStateSerializer;
+import org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumerator;
+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.reader.MongoSourceReader;
+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.split.MongoSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+import org.bson.BsonDocument;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+import java.util.function.Supplier;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The Source implementation of MongoDB. Please use a {@link MongoSourceBuilder} to construct a

Review Comment:
   ```suggestion
    * The Source implementation of MongoDB. Use a {@link MongoSourceBuilder} to construct a
   ```



##########
pom.xml:
##########
@@ -0,0 +1,457 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16-SNAPSHOT</flink.version>

Review Comment:
   ```suggestion
   		<flink.version>1.16.0</flink.version>
   ```



##########
pom.xml:
##########
@@ -0,0 +1,457 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16-SNAPSHOT</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<hamcrest.version>1.3</hamcrest.version>
+		<assertj.version>3.21.0</assertj.version>
+		<archunit.version>0.22.0</archunit.version>
+		<testcontainers.version>1.17.2</testcontainers.version>
+		<mockito.version>3.4.6</mockito.version>
+
+		<japicmp.skip>false</japicmp.skip>
+		<japicmp.referenceVersion>1.15.0</japicmp.referenceVersion>
+
+		<slf4j.version>1.7.36</slf4j.version>
+		<log4j.version>2.17.2</log4j.version>
+
+		<flink.parent.artifactId>flink-connector-mongodb-parent</flink.parent.artifactId>
+	</properties>
+
+	<modules>
+		<module>flink-connector-mongodb</module>
+		<module>flink-sql-connector-mongodb</module>
+		<module>flink-connector-mongodb-e2e-tests</module>
+	</modules>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-force-shading</artifactId>
+			<version>${flink.shaded.version}</version>
+		</dependency>
+
+		<!-- Root dependencies for all projects -->
+
+		<!-- Logging API -->
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- 'javax.annotation' classes like '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.vintage</groupId>
+			<artifactId>junit-vintage-engine</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.assertj</groupId>
+			<artifactId>assertj-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.hamcrest</groupId>
+			<artifactId>hamcrest-all</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-inline</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-core</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Tests will have log4j as the default logging framework available -->
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-slf4j-impl</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- API bridge between log4j 1 and 2 -->
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-1.2-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-production</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.mongodb</groupId>
+				<artifactId>mongodb-driver-sync</artifactId>
+				<version>${mongodb.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-api</artifactId>
+					</exclusion>
+				</exclusions>

Review Comment:
   ```suggestion
   ```
   There isn't really a reason to exclude this.



##########
pom.xml:
##########
@@ -0,0 +1,457 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16-SNAPSHOT</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<hamcrest.version>1.3</hamcrest.version>
+		<assertj.version>3.21.0</assertj.version>
+		<archunit.version>0.22.0</archunit.version>
+		<testcontainers.version>1.17.2</testcontainers.version>
+		<mockito.version>3.4.6</mockito.version>
+
+		<japicmp.skip>false</japicmp.skip>
+		<japicmp.referenceVersion>1.15.0</japicmp.referenceVersion>
+
+		<slf4j.version>1.7.36</slf4j.version>
+		<log4j.version>2.17.2</log4j.version>
+
+		<flink.parent.artifactId>flink-connector-mongodb-parent</flink.parent.artifactId>
+	</properties>
+
+	<modules>
+		<module>flink-connector-mongodb</module>
+		<module>flink-sql-connector-mongodb</module>
+		<module>flink-connector-mongodb-e2e-tests</module>
+	</modules>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-force-shading</artifactId>
+			<version>${flink.shaded.version}</version>
+		</dependency>
+
+		<!-- Root dependencies for all projects -->
+
+		<!-- Logging API -->
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- 'javax.annotation' classes like '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.vintage</groupId>
+			<artifactId>junit-vintage-engine</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.assertj</groupId>
+			<artifactId>assertj-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.hamcrest</groupId>
+			<artifactId>hamcrest-all</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-inline</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-core</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Tests will have log4j as the default logging framework available -->
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-slf4j-impl</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- API bridge between log4j 1 and 2 -->
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-1.2-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-production</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.mongodb</groupId>
+				<artifactId>mongodb-driver-sync</artifactId>
+				<version>${mongodb.version}</version>
+				<exclusions>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-api</artifactId>
+					</exclusion>
+				</exclusions>
+			</dependency>
+
+			<!-- Flink dependencies -->
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-test-utils</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+				<exclusions>
+					<exclusion>
+						<groupId>log4j</groupId>
+						<artifactId>log4j</artifactId>
+					</exclusion>
+					<exclusion>
+						<groupId>org.slf4j</groupId>
+						<artifactId>slf4j-log4j12</artifactId>
+					</exclusion>
+				</exclusions>

Review Comment:
   ```suggestion
   ```
   With 1.16.0 these _should_ no longer leak through.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.architecture;
+
+import org.apache.flink.architecture.common.ImportOptions;
+
+import com.tngtech.archunit.core.importer.ImportOption;
+import com.tngtech.archunit.junit.AnalyzeClasses;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.junit.ArchTests;
+
+/** Architecture tests for test code. */
+@AnalyzeClasses(
+        packages = "org.apache.flink.connector.mongodb",
+        importOptions = {
+            ImportOption.OnlyIncludeTests.class,
+            ImportOptions.ExcludeScalaImportOption.class,
+            ImportOptions.ExcludeShadedImportOption.class
+        })
+public class TestCodeArchitectureTest {
+
+    @ArchTest
+    public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
+}

Review Comment:
   There should also be a `ProductionCodeArchitectureTest`



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumerator.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.SplitEnumerator;
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.apache.flink.connector.mongodb.source.MongoSource;
+import org.apache.flink.connector.mongodb.source.enumerator.assigner.MongoSplitAssigner;
+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.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.TreeSet;
+
+/** The enumerator class for {@link MongoSource}. */
+@Internal
+public class MongoSourceEnumerator
+        implements SplitEnumerator<MongoSourceSplit, MongoSourceEnumState> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSourceEnumerator.class);
+
+    private final Boundedness boundedness;
+    private final SplitEnumeratorContext<MongoSourceSplit> context;
+    private final MongoSplitAssigner splitAssigner;
+    private final TreeSet<Integer> readersAwaitingSplit;
+
+    public MongoSourceEnumerator(
+            Boundedness boundedness,
+            SplitEnumeratorContext<MongoSourceSplit> context,
+            MongoSplitAssigner splitAssigner) {
+        this.boundedness = boundedness;
+        this.context = context;
+        this.splitAssigner = splitAssigner;
+        this.readersAwaitingSplit = new TreeSet<>();
+    }
+
+    @Override
+    public void start() {
+        splitAssigner.open();
+    }
+
+    @Override
+    public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) {
+        if (!context.registeredReaders().containsKey(subtaskId)) {
+            // reader failed between sending the request and now. skip this request.
+            return;
+        }
+
+        readersAwaitingSplit.add(subtaskId);
+        assignSplits();
+    }
+
+    @Override
+    public void addSplitsBack(List<MongoSourceSplit> splits, int subtaskId) {
+        LOG.debug("Mongo Source Enumerator adds splits back: {}", splits);
+        splitAssigner.addSplitsBack(splits);
+    }
+
+    @Override
+    public void addReader(int subtaskId) {
+        LOG.debug("Adding reader {} to MongoSourceEnumerator.", subtaskId);

Review Comment:
   Why isnt this doing anything?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_FETCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SAMPLES;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_STRATEGY;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The configuration class for MongoDB source. */
+@PublicEvolving
+public class MongoReadOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int fetchSize;
+
+    private final int cursorBatchSize;
+
+    private final boolean noCursorTimeout;
+
+    private final PartitionStrategy partitionStrategy;
+
+    private final MemorySize partitionSize;
+
+    private final int samplesPerPartition;
+
+    private MongoReadOptions(
+            int fetchSize,
+            int cursorBatchSize,
+            boolean noCursorTimeout,
+            PartitionStrategy partitionStrategy,
+            MemorySize partitionSize,
+            int samplesPerPartition) {
+        this.fetchSize = fetchSize;
+        this.cursorBatchSize = cursorBatchSize;
+        this.noCursorTimeout = noCursorTimeout;
+        this.partitionStrategy = partitionStrategy;
+        this.partitionSize = partitionSize;
+        this.samplesPerPartition = samplesPerPartition;
+    }
+
+    public int getFetchSize() {
+        return fetchSize;
+    }
+
+    public int getCursorBatchSize() {
+        return cursorBatchSize;
+    }
+
+    public boolean isNoCursorTimeout() {
+        return noCursorTimeout;
+    }
+
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    public MemorySize getPartitionSize() {
+        return partitionSize;
+    }
+
+    public int getSamplesPerPartition() {
+        return samplesPerPartition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoReadOptions that = (MongoReadOptions) o;
+        return cursorBatchSize == that.cursorBatchSize
+                && noCursorTimeout == that.noCursorTimeout
+                && partitionStrategy == that.partitionStrategy
+                && samplesPerPartition == that.samplesPerPartition
+                && Objects.equals(partitionSize, that.partitionSize);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                cursorBatchSize,
+                noCursorTimeout,
+                partitionStrategy,
+                partitionSize,
+                samplesPerPartition);
+    }
+
+    public static MongoReadOptionsBuilder builder() {
+        return new MongoReadOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoReadOptions}. */
+    public static class MongoReadOptionsBuilder {
+        private int fetchSize = SCAN_FETCH_SIZE.defaultValue();
+        private int cursorBatchSize = SCAN_CURSOR_BATCH_SIZE.defaultValue();
+        private boolean noCursorTimeout = SCAN_CURSOR_NO_TIMEOUT.defaultValue();
+        private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue();
+        private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue();
+        private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue();
+
+        /**
+         * Sets the number of documents should be fetched per round-trip when reading.
+         *
+         * @param fetchSize the number of documents should be fetched per round-trip when reading.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setFetchSize(int fetchSize) {
+            checkArgument(fetchSize > 0, "The fetch size must be larger than 0.");
+            this.fetchSize = fetchSize;
+            return this;
+        }
+
+        /**
+         * Sets the batch size of MongoDB find cursor.
+         *
+         * @param cursorBatchSize the max batch size of find cursor.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setCursorBatchSize(int cursorBatchSize) {
+            checkArgument(
+                    cursorBatchSize >= 0,
+                    "The cursor batch size must be larger than or equal to 0.");
+            this.cursorBatchSize = cursorBatchSize;
+            return this;
+        }
+
+        /**
+         * Set this option to true to prevent cursor timeout (10 minutes).
+         *
+         * @param noCursorTimeout Set this option to true to prevent cursor timeout (10 minutes)
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setNoCursorTimeout(boolean noCursorTimeout) {
+            this.noCursorTimeout = noCursorTimeout;
+            return this;
+        }
+
+        /**
+         * Sets the partition strategy.
+         *
+         * @param partitionStrategy the strategy of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionStrategy(PartitionStrategy partitionStrategy) {
+            checkNotNull(partitionStrategy, "The partition strategy must not be null.");
+            this.partitionStrategy = partitionStrategy;
+            return this;
+        }
+
+        /**
+         * Sets the partition size of MongoDB split.
+         *
+         * @param partitionSize the memory size of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionSize(MemorySize partitionSize) {
+            checkNotNull(partitionSize, "The partition size must not be null");
+            checkArgument(
+                    partitionSize.getMebiBytes() >= 1,
+                    "The partition size must be larger than or equals to 1mb.");

Review Comment:
   why?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoConstants.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonMaxKey;
+import org.bson.BsonMinKey;
+import org.bson.BsonValue;
+
+/** Constants for MongoDB. */
+@PublicEvolving

Review Comment:
   How are these constants used by users?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SINK_MAX_RETRIES;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The configured class for Mongo sink. */
+@PublicEvolving
+public class MongoWriteOptions implements Serializable {

Review Comment:
   ```suggestion
   public final class MongoWriteOptions implements Serializable {
   ```
   Apply to other classes where it makes sense.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_FETCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SAMPLES;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_STRATEGY;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The configuration class for MongoDB source. */
+@PublicEvolving
+public class MongoReadOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int fetchSize;
+
+    private final int cursorBatchSize;
+
+    private final boolean noCursorTimeout;
+
+    private final PartitionStrategy partitionStrategy;
+
+    private final MemorySize partitionSize;
+
+    private final int samplesPerPartition;
+
+    private MongoReadOptions(
+            int fetchSize,
+            int cursorBatchSize,
+            boolean noCursorTimeout,
+            PartitionStrategy partitionStrategy,
+            MemorySize partitionSize,
+            int samplesPerPartition) {
+        this.fetchSize = fetchSize;
+        this.cursorBatchSize = cursorBatchSize;
+        this.noCursorTimeout = noCursorTimeout;
+        this.partitionStrategy = partitionStrategy;
+        this.partitionSize = partitionSize;
+        this.samplesPerPartition = samplesPerPartition;
+    }
+
+    public int getFetchSize() {
+        return fetchSize;
+    }
+
+    public int getCursorBatchSize() {
+        return cursorBatchSize;
+    }
+
+    public boolean isNoCursorTimeout() {
+        return noCursorTimeout;
+    }
+
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    public MemorySize getPartitionSize() {
+        return partitionSize;
+    }
+
+    public int getSamplesPerPartition() {
+        return samplesPerPartition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoReadOptions that = (MongoReadOptions) o;
+        return cursorBatchSize == that.cursorBatchSize
+                && noCursorTimeout == that.noCursorTimeout
+                && partitionStrategy == that.partitionStrategy
+                && samplesPerPartition == that.samplesPerPartition
+                && Objects.equals(partitionSize, that.partitionSize);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                cursorBatchSize,
+                noCursorTimeout,
+                partitionStrategy,
+                partitionSize,
+                samplesPerPartition);
+    }
+
+    public static MongoReadOptionsBuilder builder() {
+        return new MongoReadOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoReadOptions}. */
+    public static class MongoReadOptionsBuilder {
+        private int fetchSize = SCAN_FETCH_SIZE.defaultValue();
+        private int cursorBatchSize = SCAN_CURSOR_BATCH_SIZE.defaultValue();
+        private boolean noCursorTimeout = SCAN_CURSOR_NO_TIMEOUT.defaultValue();
+        private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue();
+        private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue();
+        private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue();
+
+        /**
+         * Sets the number of documents should be fetched per round-trip when reading.
+         *
+         * @param fetchSize the number of documents should be fetched per round-trip when reading.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setFetchSize(int fetchSize) {
+            checkArgument(fetchSize > 0, "The fetch size must be larger than 0.");
+            this.fetchSize = fetchSize;
+            return this;
+        }
+
+        /**
+         * Sets the batch size of MongoDB find cursor.
+         *
+         * @param cursorBatchSize the max batch size of find cursor.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setCursorBatchSize(int cursorBatchSize) {
+            checkArgument(
+                    cursorBatchSize >= 0,
+                    "The cursor batch size must be larger than or equal to 0.");
+            this.cursorBatchSize = cursorBatchSize;
+            return this;
+        }
+
+        /**
+         * Set this option to true to prevent cursor timeout (10 minutes).
+         *
+         * @param noCursorTimeout Set this option to true to prevent cursor timeout (10 minutes)
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setNoCursorTimeout(boolean noCursorTimeout) {
+            this.noCursorTimeout = noCursorTimeout;
+            return this;
+        }
+
+        /**
+         * Sets the partition strategy.
+         *
+         * @param partitionStrategy the strategy of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionStrategy(PartitionStrategy partitionStrategy) {
+            checkNotNull(partitionStrategy, "The partition strategy must not be null.");
+            this.partitionStrategy = partitionStrategy;
+            return this;
+        }
+
+        /**
+         * Sets the partition size of MongoDB split.
+         *
+         * @param partitionSize the memory size of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionSize(MemorySize partitionSize) {
+            checkNotNull(partitionSize, "The partition size must not be null");
+            checkArgument(
+                    partitionSize.getMebiBytes() >= 1,
+                    "The partition size must be larger than or equals to 1mb.");
+            this.partitionSize = partitionSize;
+            return this;
+        }
+
+        /**
+         * Sets the partition size of MongoDB split.

Review Comment:
   javadocs are identical to setPartitionSize



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContextImpl;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible to write records in a MongoDB collection.

Review Comment:
   ```suggestion
    * This class is responsible for writing records to a MongoDB collection.
   ```



##########
pom.xml:
##########
@@ -0,0 +1,457 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16-SNAPSHOT</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<hamcrest.version>1.3</hamcrest.version>
+		<assertj.version>3.21.0</assertj.version>
+		<archunit.version>0.22.0</archunit.version>
+		<testcontainers.version>1.17.2</testcontainers.version>
+		<mockito.version>3.4.6</mockito.version>
+
+		<japicmp.skip>false</japicmp.skip>
+		<japicmp.referenceVersion>1.15.0</japicmp.referenceVersion>
+
+		<slf4j.version>1.7.36</slf4j.version>
+		<log4j.version>2.17.2</log4j.version>
+
+		<flink.parent.artifactId>flink-connector-mongodb-parent</flink.parent.artifactId>
+	</properties>
+
+	<modules>
+		<module>flink-connector-mongodb</module>
+		<module>flink-sql-connector-mongodb</module>
+		<module>flink-connector-mongodb-e2e-tests</module>
+	</modules>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-force-shading</artifactId>
+			<version>${flink.shaded.version}</version>
+		</dependency>
+
+		<!-- Root dependencies for all projects -->
+
+		<!-- Logging API -->
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- 'javax.annotation' classes like '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.vintage</groupId>
+			<artifactId>junit-vintage-engine</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.assertj</groupId>
+			<artifactId>assertj-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.hamcrest</groupId>
+			<artifactId>hamcrest-all</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-inline</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-core</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Tests will have log4j as the default logging framework available -->
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-slf4j-impl</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- API bridge between log4j 1 and 2 -->
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-1.2-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-production</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.mongodb</groupId>
+				<artifactId>mongodb-driver-sync</artifactId>

Review Comment:
   Why did you choose this client instead of the Reactive Streams Driver?
   Isn't the sync client potentially blocking the source/sink thread for a long time?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SINK_MAX_RETRIES;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The configured class for Mongo sink. */
+@PublicEvolving
+public class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final @Nullable Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private @Nullable Integer parallelism;
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkState(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkState(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0, "The max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#NONE}

Review Comment:
   The default shouldn't drop data.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSourceBuilder.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema;
+
+import org.bson.BsonDocument;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.flink.util.CollectionUtil.isNullOrEmpty;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The builder class for {@link MongoSource} to make it easier for the users to construct a {@link
+ * MongoSource}.
+ *
+ * @param <OUT> The output type of the source.
+ */
+@PublicEvolving
+public class MongoSourceBuilder<OUT> {
+
+    private final MongoConnectionOptions.MongoConnectionOptionsBuilder connectionOptionsBuilder;
+    private final MongoReadOptions.MongoReadOptionsBuilder readOptionsBuilder;
+
+    private List<String> projectedFields;
+    private int limit = -1;
+    private MongoDeserializationSchema<OUT> deserializationSchema;
+
+    MongoSourceBuilder() {
+        this.connectionOptionsBuilder = MongoConnectionOptions.builder();
+        this.readOptionsBuilder = MongoReadOptions.builder();
+    }
+
+    /**
+     * Sets the connection string of MongoDB.
+     *
+     * @param uri connection string of MongoDB
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setUri(String uri) {
+        connectionOptionsBuilder.setUri(uri);
+        return this;
+    }
+
+    /**
+     * Sets the database to sink of MongoDB.
+     *
+     * @param database the database to sink of MongoDB.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setDatabase(String database) {
+        connectionOptionsBuilder.setDatabase(database);
+        return this;
+    }
+
+    /**
+     * Sets the collection to sink of MongoDB.
+     *
+     * @param collection the collection to sink of MongoDB.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setCollection(String collection) {
+        connectionOptionsBuilder.setCollection(collection);
+        return this;
+    }
+
+    /**
+     * Sets the number of documents should be fetched per round-trip when reading.
+     *
+     * @param fetchSize the number of documents should be fetched per round-trip when reading.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setFetchSize(int fetchSize) {
+        readOptionsBuilder.setFetchSize(fetchSize);
+        return this;
+    }
+
+    /**
+     * Sets the batch size of MongoDB find cursor.
+     *
+     * @param cursorBatchSize the max batch size of find cursor.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setCursorBatchSize(int cursorBatchSize) {
+        readOptionsBuilder.setCursorBatchSize(cursorBatchSize);
+        return this;
+    }
+
+    /**
+     * Set this option to true to prevent cursor timeout (defaults to 10 minutes).
+     *
+     * @param noCursorTimeout Set this option to true to prevent cursor timeout.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setNoCursorTimeout(boolean noCursorTimeout) {
+        readOptionsBuilder.setNoCursorTimeout(noCursorTimeout);
+        return this;
+    }
+
+    /**
+     * Sets the partition strategy.
+     *
+     * @param partitionStrategy the strategy of a partition.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setPartitionStrategy(PartitionStrategy partitionStrategy) {
+        readOptionsBuilder.setPartitionStrategy(partitionStrategy);
+        return this;
+    }
+
+    /**
+     * Sets the partition size of MongoDB split.
+     *
+     * @param partitionSize the memory size of a partition.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setPartitionSize(MemorySize partitionSize) {
+        readOptionsBuilder.setPartitionSize(partitionSize);
+        return this;
+    }
+
+    /**
+     * Sets the samples size per partition only effective for sample partition strategy.
+     *
+     * @param samplesPerPartition the samples size per partition
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setSamplesPerPartition(int samplesPerPartition) {
+        readOptionsBuilder.setSamplesPerPartition(samplesPerPartition);
+        return this;
+    }
+
+    /**
+     * Sets the limit of documents to read.

Review Comment:
   What happens if limit is not set?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContextImpl;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible to write records in a MongoDB collection.
+ *
+ * @param <IN> The type of the input elements.
+ */
+@Internal
+public class MongoWriter<IN> implements SinkWriter<IN> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriter.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+    private final MongoSinkContext sinkContext;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean flushOnCheckpoint;
+    private final List<WriteModel<BsonDocument>> bulkRequests = new ArrayList<>();
+    private final Collector<WriteModel<BsonDocument>> collector;
+    private final MongoClient mongoClient;
+
+    private boolean checkpointInProgress = false;
+    private volatile long lastSendTime = 0L;
+    private volatile long ackTime = Long.MAX_VALUE;
+
+    public MongoWriter(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            boolean flushOnCheckpoint,
+            Sink.InitContext initContext,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);
+        this.flushOnCheckpoint = flushOnCheckpoint;
+
+        checkNotNull(initContext);
+        this.mailboxExecutor = checkNotNull(initContext.getMailboxExecutor());
+
+        SinkWriterMetricGroup metricGroup = checkNotNull(initContext.metricGroup());
+        metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+
+        this.collector =
+                new CountingCollector<>(
+                        new ListCollector<>(this.bulkRequests),
+                        metricGroup.getNumRecordsSendCounter());
+
+        // Initialize the serialization schema.
+        this.sinkContext = new MongoSinkContextImpl(initContext, writeOptions);
+        try {
+            SerializationSchema.InitializationContext initializationContext =
+                    initContext.asSerializationSchemaInitializationContext();
+            serializationSchema.open(initializationContext, sinkContext, writeOptions);
+        } catch (Exception e) {
+            throw new FlinkRuntimeException("Failed to open the MongoEmitter", e);
+        }
+
+        // Initialize the mongo client.
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    @Override
+    public void write(IN element, Context context) throws IOException, InterruptedException {
+        // do not allow new bulk writes until all actions are flushed
+        while (checkpointInProgress) {
+            mailboxExecutor.yield();
+        }
+        collector.collect(serializationSchema.serialize(element, sinkContext));
+        if (isOverMaxActionsLimit() || isOverMaxFlushIntervalLimit()) {
+            doBulkWrite();
+        }
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException {
+        checkpointInProgress = true;
+        while (!bulkRequests.isEmpty() && (flushOnCheckpoint || endOfInput)) {
+            doBulkWrite();
+        }
+        checkpointInProgress = false;
+    }
+
+    @Override
+    public void close() {
+        if (mongoClient != null) {

Review Comment:
   This condition is always true.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/context/MongoSinkContextImpl.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.sink.writer.context;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+
+/** An implementation that would contain all the required context. */
+@Internal
+public class MongoSinkContextImpl implements MongoSinkContext {

Review Comment:
   ```suggestion
   public class DefaultMongoSinkContext implements MongoSinkContext {
   ```
   It's usually more consistent to add such a qualifier to the front of the class name.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom

Review Comment:
   How's that supposed to work?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContextImpl;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible to write records in a MongoDB collection.
+ *
+ * @param <IN> The type of the input elements.
+ */
+@Internal
+public class MongoWriter<IN> implements SinkWriter<IN> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriter.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+    private final MongoSinkContext sinkContext;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean flushOnCheckpoint;
+    private final List<WriteModel<BsonDocument>> bulkRequests = new ArrayList<>();
+    private final Collector<WriteModel<BsonDocument>> collector;
+    private final MongoClient mongoClient;
+
+    private boolean checkpointInProgress = false;
+    private volatile long lastSendTime = 0L;
+    private volatile long ackTime = Long.MAX_VALUE;
+
+    public MongoWriter(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            boolean flushOnCheckpoint,
+            Sink.InitContext initContext,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);
+        this.flushOnCheckpoint = flushOnCheckpoint;
+
+        checkNotNull(initContext);
+        this.mailboxExecutor = checkNotNull(initContext.getMailboxExecutor());
+
+        SinkWriterMetricGroup metricGroup = checkNotNull(initContext.metricGroup());
+        metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+
+        this.collector =
+                new CountingCollector<>(
+                        new ListCollector<>(this.bulkRequests),
+                        metricGroup.getNumRecordsSendCounter());
+
+        // Initialize the serialization schema.
+        this.sinkContext = new MongoSinkContextImpl(initContext, writeOptions);
+        try {
+            SerializationSchema.InitializationContext initializationContext =
+                    initContext.asSerializationSchemaInitializationContext();
+            serializationSchema.open(initializationContext, sinkContext, writeOptions);
+        } catch (Exception e) {
+            throw new FlinkRuntimeException("Failed to open the MongoEmitter", e);
+        }
+
+        // Initialize the mongo client.
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    @Override
+    public void write(IN element, Context context) throws IOException, InterruptedException {
+        // do not allow new bulk writes until all actions are flushed
+        while (checkpointInProgress) {
+            mailboxExecutor.yield();
+        }
+        collector.collect(serializationSchema.serialize(element, sinkContext));
+        if (isOverMaxActionsLimit() || isOverMaxFlushIntervalLimit()) {
+            doBulkWrite();
+        }
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException {
+        checkpointInProgress = true;
+        while (!bulkRequests.isEmpty() && (flushOnCheckpoint || endOfInput)) {
+            doBulkWrite();
+        }
+        checkpointInProgress = false;
+    }
+
+    @Override
+    public void close() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @VisibleForTesting
+    void doBulkWrite() throws IOException {
+        if (bulkRequests.isEmpty()) {
+            // no records to write
+            return;
+        }
+
+        int maxRetryTimes = writeOptions.getMaxRetryTimes();
+        for (int i = 0; i <= maxRetryTimes; i++) {
+            try {
+                lastSendTime = System.currentTimeMillis();
+                mongoClient
+                        .getDatabase(connectionOptions.getDatabase())
+                        .getCollection(connectionOptions.getCollection(), BsonDocument.class)
+                        .bulkWrite(bulkRequests);
+                ackTime = System.currentTimeMillis();
+                bulkRequests.clear();
+                break;
+            } catch (MongoException e) {
+                LOG.error("Bulk Write to MongoDB failed, retry times = {}", i, e);
+                if (i >= maxRetryTimes) {
+                    throw new IOException(e);
+                }
+                try {
+                    Thread.sleep(1000L * i);

Review Comment:
   this should be configurable.
   
   The first attempt also doesn't wait at all which is likely not intended.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/context/MongoSinkContextImpl.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.sink.writer.context;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+
+/** An implementation that would contain all the required context. */

Review Comment:
   ```suggestion
   /** Default {@link MongoSinkContext} implementation. */
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SINK_MAX_RETRIES;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The configured class for Mongo sink. */
+@PublicEvolving
+public class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final @Nullable Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private @Nullable Integer parallelism;
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkState(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkState(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0, "The max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#NONE}
+         *
+         * @param deliveryGuarantee which describes the record emission behaviour
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+            checkState(
+                    deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+                    "Mongo sink does not support the EXACTLY_ONCE guarantee.");
+            this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+            return this;
+        }
+
+        /**
+         * Sets the write parallelism.

Review Comment:
   This needs a better description. Is it something like maxConcurrentBulks? Why should you use this over increasing the sink parallelism?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,205 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SINK_MAX_RETRIES;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The configured class for Mongo sink. */
+@PublicEvolving
+public class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final @Nullable Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private @Nullable Integer parallelism;
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkState(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkState(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0, "The max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#NONE}
+         *
+         * @param deliveryGuarantee which describes the record emission behaviour
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+            checkState(
+                    deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+                    "Mongo sink does not support the EXACTLY_ONCE guarantee.");
+            this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+            return this;
+        }
+
+        /**
+         * Sets the write parallelism.
+         *
+         * @param parallelism the write parallelism
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setParallelism(Integer parallelism) {
+            checkArgument(
+                    parallelism == null || parallelism > 0,
+                    "Mongo sink parallelism must be larger than 0.");
+            this.parallelism = parallelism;
+            return this;
+        }

Review Comment:
   ```suggestion
           public MongoWriteOptionsBuilder setParallelism(int parallelism) {
               checkArgument(parallelism > 0,
                       "Mongo sink parallelism must be larger than 0.");
               this.parallelism = parallelism;
               return this;
           }
   ```
   use an int



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom
+     *     options.
+     */
+    default void open(
+            SerializationSchema.InitializationContext initializationContext,
+            MongoSinkContext sinkContext,
+            MongoWriteOptions sinkConfiguration)

Review Comment:
   What are the use-cases for passing the sinkConfiguration?



##########
flink-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,165 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0.0-SNAPSHOT</version>
+		<relativePath>../pom.xml</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-mongodb</artifactId>
+	<name>Flink : Connectors : MongoDB</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<!-- Core -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-base</artifactId>
+			<version>${flink.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${flink.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- MongoDB -->
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+		</dependency>
+
+		<!-- Table ecosystem -->
+
+		<!-- Projects depending on this project won't depend on flink-table-*. -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-api-java-bridge</artifactId>
+			<version>${flink.version}</version>
+			<scope>provided</scope>
+			<optional>true</optional>
+		</dependency>
+
+		<!-- Tests -->
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime</artifactId>
+			<version>${flink.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${flink.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Table API integration tests -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-planner-loader</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-runtime</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-common</artifactId>
+			<version>${flink.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-base</artifactId>
+			<version>${flink.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>

Review Comment:
   who is the intended consumer for the test-jar, and for what purpose?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContextImpl;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible to write records in a MongoDB collection.
+ *
+ * @param <IN> The type of the input elements.
+ */
+@Internal
+public class MongoWriter<IN> implements SinkWriter<IN> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriter.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+    private final MongoSinkContext sinkContext;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean flushOnCheckpoint;
+    private final List<WriteModel<BsonDocument>> bulkRequests = new ArrayList<>();
+    private final Collector<WriteModel<BsonDocument>> collector;
+    private final MongoClient mongoClient;
+
+    private boolean checkpointInProgress = false;
+    private volatile long lastSendTime = 0L;
+    private volatile long ackTime = Long.MAX_VALUE;
+
+    public MongoWriter(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            boolean flushOnCheckpoint,
+            Sink.InitContext initContext,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);
+        this.flushOnCheckpoint = flushOnCheckpoint;
+
+        checkNotNull(initContext);
+        this.mailboxExecutor = checkNotNull(initContext.getMailboxExecutor());
+
+        SinkWriterMetricGroup metricGroup = checkNotNull(initContext.metricGroup());
+        metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+
+        this.collector =
+                new CountingCollector<>(
+                        new ListCollector<>(this.bulkRequests),
+                        metricGroup.getNumRecordsSendCounter());
+
+        // Initialize the serialization schema.
+        this.sinkContext = new MongoSinkContextImpl(initContext, writeOptions);
+        try {
+            SerializationSchema.InitializationContext initializationContext =
+                    initContext.asSerializationSchemaInitializationContext();
+            serializationSchema.open(initializationContext, sinkContext, writeOptions);
+        } catch (Exception e) {
+            throw new FlinkRuntimeException("Failed to open the MongoEmitter", e);
+        }
+
+        // Initialize the mongo client.
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    @Override
+    public void write(IN element, Context context) throws IOException, InterruptedException {
+        // do not allow new bulk writes until all actions are flushed
+        while (checkpointInProgress) {
+            mailboxExecutor.yield();
+        }
+        collector.collect(serializationSchema.serialize(element, sinkContext));
+        if (isOverMaxActionsLimit() || isOverMaxFlushIntervalLimit()) {
+            doBulkWrite();
+        }
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException {
+        checkpointInProgress = true;
+        while (!bulkRequests.isEmpty() && (flushOnCheckpoint || endOfInput)) {
+            doBulkWrite();
+        }
+        checkpointInProgress = false;
+    }
+
+    @Override
+    public void close() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @VisibleForTesting
+    void doBulkWrite() throws IOException {
+        if (bulkRequests.isEmpty()) {
+            // no records to write
+            return;
+        }
+
+        int maxRetryTimes = writeOptions.getMaxRetryTimes();
+        for (int i = 0; i <= maxRetryTimes; i++) {
+            try {
+                lastSendTime = System.currentTimeMillis();
+                mongoClient
+                        .getDatabase(connectionOptions.getDatabase())
+                        .getCollection(connectionOptions.getCollection(), BsonDocument.class)
+                        .bulkWrite(bulkRequests);
+                ackTime = System.currentTimeMillis();
+                bulkRequests.clear();
+                break;
+            } catch (MongoException e) {
+                LOG.error("Bulk Write to MongoDB failed, retry times = {}", i, e);

Review Comment:
   This shouldn't be logged as an error; retries aren't unusual. Only log an error if it fails completely.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011724216


##########
flink-connector-mongodb/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.architecture;
+
+import org.apache.flink.architecture.common.ImportOptions;
+
+import com.tngtech.archunit.core.importer.ImportOption;
+import com.tngtech.archunit.junit.AnalyzeClasses;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.junit.ArchTests;
+
+/** Architecture tests for test code. */
+@AnalyzeClasses(
+        packages = "org.apache.flink.connector.mongodb",
+        importOptions = {
+            ImportOption.OnlyIncludeTests.class,
+            ImportOptions.ExcludeScalaImportOption.class,
+            ImportOptions.ExcludeShadedImportOption.class
+        })
+public class TestCodeArchitectureTest {
+
+    @ArchTest
+    public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
+}

Review Comment:
   Thanks, architecture test for production code added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1016658710


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom

Review Comment:
   Thanks @zentol, I think we can delete `You can add custom` descriptions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010626817


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/config/MongoReadOptions.java:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_FETCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SAMPLES;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_SIZE;
+import static org.apache.flink.connector.mongodb.table.config.MongoConnectorOptions.SCAN_PARTITION_STRATEGY;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The configuration class for MongoDB source. */
+@PublicEvolving
+public class MongoReadOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int fetchSize;
+
+    private final int cursorBatchSize;
+
+    private final boolean noCursorTimeout;
+
+    private final PartitionStrategy partitionStrategy;
+
+    private final MemorySize partitionSize;
+
+    private final int samplesPerPartition;
+
+    private MongoReadOptions(
+            int fetchSize,
+            int cursorBatchSize,
+            boolean noCursorTimeout,
+            PartitionStrategy partitionStrategy,
+            MemorySize partitionSize,
+            int samplesPerPartition) {
+        this.fetchSize = fetchSize;
+        this.cursorBatchSize = cursorBatchSize;
+        this.noCursorTimeout = noCursorTimeout;
+        this.partitionStrategy = partitionStrategy;
+        this.partitionSize = partitionSize;
+        this.samplesPerPartition = samplesPerPartition;
+    }
+
+    public int getFetchSize() {
+        return fetchSize;
+    }
+
+    public int getCursorBatchSize() {
+        return cursorBatchSize;
+    }
+
+    public boolean isNoCursorTimeout() {
+        return noCursorTimeout;
+    }
+
+    public PartitionStrategy getPartitionStrategy() {
+        return partitionStrategy;
+    }
+
+    public MemorySize getPartitionSize() {
+        return partitionSize;
+    }
+
+    public int getSamplesPerPartition() {
+        return samplesPerPartition;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoReadOptions that = (MongoReadOptions) o;
+        return cursorBatchSize == that.cursorBatchSize
+                && noCursorTimeout == that.noCursorTimeout
+                && partitionStrategy == that.partitionStrategy
+                && samplesPerPartition == that.samplesPerPartition
+                && Objects.equals(partitionSize, that.partitionSize);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                cursorBatchSize,
+                noCursorTimeout,
+                partitionStrategy,
+                partitionSize,
+                samplesPerPartition);
+    }
+
+    public static MongoReadOptionsBuilder builder() {
+        return new MongoReadOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoReadOptions}. */
+    public static class MongoReadOptionsBuilder {
+        private int fetchSize = SCAN_FETCH_SIZE.defaultValue();
+        private int cursorBatchSize = SCAN_CURSOR_BATCH_SIZE.defaultValue();
+        private boolean noCursorTimeout = SCAN_CURSOR_NO_TIMEOUT.defaultValue();
+        private PartitionStrategy partitionStrategy = SCAN_PARTITION_STRATEGY.defaultValue();
+        private MemorySize partitionSize = SCAN_PARTITION_SIZE.defaultValue();
+        private int samplesPerPartition = SCAN_PARTITION_SAMPLES.defaultValue();
+
+        /**
+         * Sets the number of documents should be fetched per round-trip when reading.
+         *
+         * @param fetchSize the number of documents should be fetched per round-trip when reading.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setFetchSize(int fetchSize) {
+            checkArgument(fetchSize > 0, "The fetch size must be larger than 0.");
+            this.fetchSize = fetchSize;
+            return this;
+        }
+
+        /**
+         * Sets the batch size of MongoDB find cursor.
+         *
+         * @param cursorBatchSize the max batch size of find cursor.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setCursorBatchSize(int cursorBatchSize) {
+            checkArgument(
+                    cursorBatchSize >= 0,
+                    "The cursor batch size must be larger than or equal to 0.");
+            this.cursorBatchSize = cursorBatchSize;
+            return this;
+        }
+
+        /**
+         * Set this option to true to prevent cursor timeout (10 minutes).
+         *
+         * @param noCursorTimeout Set this option to true to prevent cursor timeout (10 minutes)
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setNoCursorTimeout(boolean noCursorTimeout) {
+            this.noCursorTimeout = noCursorTimeout;
+            return this;
+        }
+
+        /**
+         * Sets the partition strategy.
+         *
+         * @param partitionStrategy the strategy of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionStrategy(PartitionStrategy partitionStrategy) {
+            checkNotNull(partitionStrategy, "The partition strategy must not be null.");
+            this.partitionStrategy = partitionStrategy;
+            return this;
+        }
+
+        /**
+         * Sets the partition size of MongoDB split.
+         *
+         * @param partitionSize the memory size of a partition.
+         * @return this builder
+         */
+        public MongoReadOptionsBuilder setPartitionSize(MemorySize partitionSize) {
+            checkNotNull(partitionSize, "The partition size must not be null");
+            checkArgument(
+                    partitionSize.getMebiBytes() >= 1,
+                    "The partition size must be larger than or equals to 1mb.");

Review Comment:
   Because the `splitVector` command's maxChunkSize minimum value was 1 mb. If we use parameters below 1mb, the command will fail. Also, chunks that are too small have no performance benefit for partitioning.
   
   help for: splitVector Internal command.
   examples:
     { splitVector : "blog.post" , keyPattern:{x:1} , min:{x:10} , max:{x:20}, maxChunkSize:200 }
     maxChunkSize unit in MBs
     May optionally specify 'maxSplitPoints' and 'maxChunkObjects' to avoid traversing the whole chunk
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1030486881


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reversedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // It behaves as append-only when no primary key is declared and no reversed _id is present.
+        // We use anonymous classes instead of lambdas for a reason here. It is
+        // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas
+        // (MSHADE-260).
+        if (!primaryKey.isPresent() && !reversedId.isPresent()) {
+            return new SerializableFunction<RowData, BsonValue>() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public BsonValue apply(RowData rowData) {
+                    return null;
+                }
+            };
+        }
+
+        if (reversedId.isPresent()) {
+            // Primary key should be declared as (_id) when the mongo reversed _id is present.
+            if (!primaryKey.isPresent()
+                    || isCompoundPrimaryKey(primaryKeyIndexes)
+                    || !primaryKeyContainsReversedId(primaryKey.get())) {
+                throw new IllegalArgumentException(
+                        "The primary key should be declared as (_id) when mongo reversed _id field is present");

Review Comment:
   That's an interesting question. We don't seem to handle this case in a special way in the ES connector; we use the `PRIMARY KEY` as the id and insert the document as is with it's _id_ field; the final behavior will depend on Elasticsearch.
   
   I don't really like that behavior because as you said it is ambiguous; we should fail early if the schema contains an _id field and it's not the sole primary key.
   
   My comment was rather about the error message. You are explicitly suggesting `(_id)` as an alternative in this case; but is that really the best option? Maybe they just made a mistake w.r.t. keys. Maybe we shouldn't even give such suggestions, and rather just describe the problem of ambiguous keys being used due to the presence of an `_id` field
   



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = "_id";
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reversedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // It behaves as append-only when no primary key is declared and no reversed _id is present.
+        // We use anonymous classes instead of lambdas for a reason here. It is
+        // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas
+        // (MSHADE-260).
+        if (!primaryKey.isPresent() && !reversedId.isPresent()) {
+            return new SerializableFunction<RowData, BsonValue>() {
+                private static final long serialVersionUID = 1L;
+
+                @Override
+                public BsonValue apply(RowData rowData) {
+                    return null;
+                }
+            };
+        }
+
+        if (reversedId.isPresent()) {
+            // Primary key should be declared as (_id) when the mongo reversed _id is present.
+            if (!primaryKey.isPresent()
+                    || isCompoundPrimaryKey(primaryKeyIndexes)
+                    || !primaryKeyContainsReversedId(primaryKey.get())) {
+                throw new IllegalArgumentException(
+                        "The primary key should be declared as (_id) when mongo reversed _id field is present");

Review Comment:
   That's an interesting question. We don't seem to handle this case in a special way in the ES connector; we use the `PRIMARY KEY` as the id and insert the document as is with it's _id_ field; the final behavior will depend on Elasticsearch.
   
   I don't really like that behavior because as you said it is ambiguous; we should fail early if the schema contains an `_id` field and it's not the sole primary key.
   
   My comment was rather about the error message. You are explicitly suggesting `(_id)` as an alternative in this case; but is that really the best option? Maybe they just made a mistake w.r.t. keys. Maybe we shouldn't even give such suggestions, and rather just describe the problem of ambiguous keys being used due to the presence of an `_id` field
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032603066


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactory.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.table.config.MongoConfiguration;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.HashSet;
+import java.util.Set;
+
+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.COLLECTION;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.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;
+
+/**
+ * Factory for creating configured instances of {@link MongoDynamicTableSource} and {@link
+ * MongoDynamicTableSink}.
+ */
+@Internal
+public class MongoDynamicTableFactory
+        implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+    public static final String IDENTIFIER = "mongodb";
+
+    @Override
+    public String factoryIdentifier() {
+        return IDENTIFIER;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> requiredOptions() {
+        final Set<ConfigOption<?>> requiredOptions = new HashSet<>();
+        requiredOptions.add(URI);
+        requiredOptions.add(DATABASE);
+        requiredOptions.add(COLLECTION);
+        return requiredOptions;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> optionalOptions() {
+        final Set<ConfigOption<?>> optionalOptions = new HashSet<>();
+        optionalOptions.add(SCAN_FETCH_SIZE);
+        optionalOptions.add(SCAN_CURSOR_BATCH_SIZE);
+        optionalOptions.add(SCAN_CURSOR_NO_TIMEOUT);
+        optionalOptions.add(SCAN_PARTITION_STRATEGY);
+        optionalOptions.add(SCAN_PARTITION_SIZE);
+        optionalOptions.add(SCAN_PARTITION_SAMPLES);
+        optionalOptions.add(BUFFER_FLUSH_MAX_ROWS);
+        optionalOptions.add(BUFFER_FLUSH_INTERVAL);
+        optionalOptions.add(DELIVERY_GUARANTEE);
+        optionalOptions.add(SINK_MAX_RETRIES);
+        optionalOptions.add(SINK_RETRY_INTERVAL);
+        optionalOptions.add(SINK_PARALLELISM);
+        optionalOptions.add(LookupOptions.CACHE_TYPE);
+        optionalOptions.add(LookupOptions.MAX_RETRIES);
+        optionalOptions.add(LOOKUP_RETRY_INTERVAL);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_MAX_ROWS);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_CACHE_MISSING_KEY);
+        return optionalOptions;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> forwardOptions() {

Review Comment:
   Yes, they should be.
   Thanks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1018079537


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitVectorSplitter.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoCommandException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.apache.commons.collections.CollectionUtils;
+import org.bson.BsonArray;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SPLIT_KEYS_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.UNAUTHORIZED_ERROR;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.splitVector;
+
+/**
+ * SplitVector Partitioner
+ *
+ * <p>Uses the SplitVector command to generate chunks for a collection. eg. <code>
+ * db.runCommand({splitVector:"inventory.products", keyPattern:{_id:1}, maxChunkSize:64})</code>
+ *
+ * <p>Requires splitVector privilege.
+ */
+@Internal
+public class MongoSplitVectorSplitter implements MongoSplitters.MongoSplitter {

Review Comment:
   Thanks @rozza to mention this point, it make sense to me.
   But it's hard to give up the advantages of `splitVector`, which is fast, evenly divided, and suitable for almost all scenarios.
   As for additional permission requirements, I think we can explain in the usage documentation. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1016890070


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/config/MongoConnectionOptions.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.common.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+
+import com.mongodb.ConnectionString;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The connection configuration class for MongoDB. */
+@PublicEvolving
+public class MongoConnectionOptions implements Serializable {
+
+    private final String uri;
+    private final String database;
+    private final String collection;
+
+    private MongoConnectionOptions(String uri, String database, String collection) {
+        this.uri = checkNotNull(uri);
+        this.database = checkNotNull(database);
+        this.collection = checkNotNull(collection);
+    }
+
+    public String getUri() {
+        return uri;
+    }
+
+    public String getDatabase() {
+        return database;
+    }
+
+    public String getCollection() {
+        return collection;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoConnectionOptions that = (MongoConnectionOptions) o;
+        return Objects.equals(uri, that.uri)
+                && Objects.equals(database, that.database)
+                && Objects.equals(collection, that.collection);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(uri, database, collection);
+    }
+
+    public static MongoConnectionOptionsBuilder builder() {
+        return new MongoConnectionOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoConnectionOptions}. */
+    @PublicEvolving
+    public static class MongoConnectionOptionsBuilder {
+        private String uri;
+        private String database;
+        private String collection;
+
+        /**
+         * Sets the connection string of MongoDB.
+         *
+         * @param uri connection string of MongoDB
+         * @return this builder
+         */
+        public MongoConnectionOptionsBuilder setUri(String uri) {
+            this.uri = new ConnectionString(uri).getConnectionString();
+            return this;

Review Comment:
   Yes, we can check here and fail fast.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017512361


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue() == 1;
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue() == 1L;
+        }
+        if (bsonValue.isString()) {
+            return Boolean.parseBoolean(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to boolean from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static byte convertToTinyInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (byte) 1 : (byte) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (byte) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (byte) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Byte.parseByte(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to tinyint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static short convertToSmallInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (short) 1 : (short) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (short) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (short) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Short.parseShort(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to smallint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static int convertToInt(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().intValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.intValue();
+            } else if (decimal128Value.isNegative()) {
+                return Integer.MIN_VALUE;
+            } else {
+                return Integer.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isDateTime()) {
+            return (int) Instant.ofEpochMilli(bsonValue.asDateTime().getValue()).getEpochSecond();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime();
+        }
+        if (bsonValue.isString()) {
+            return Integer.parseInt(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to integer from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static long convertToLong(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().longValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.longValue();
+            } else if (decimal128Value.isNegative()) {
+                return Long.MIN_VALUE;
+            } else {
+                return Long.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1L : 0L;
+        }
+        if (bsonValue.isDateTime()) {
+            return bsonValue.asDateTime().getValue();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime() * 1000L;
+        }
+        if (bsonValue.isString()) {
+            return Long.parseLong(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to long from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static double convertToDouble(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().doubleValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.doubleValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Double.MAX_VALUE;
+            } else {
+                return Double.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isString()) {
+            return Double.parseDouble(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to double from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static float convertToFloat(BsonValue bsonValue) {
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isDouble()) {
+            return ((Double) bsonValue.asDouble().getValue()).floatValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.floatValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Float.MAX_VALUE;
+            } else {
+                return Float.MAX_VALUE;

Review Comment:
   Fixed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#issuecomment-1301782871

   Hi @zentol.
   Sorry to bother you again.
   Please help to see if there is anything else that needs to be modified.
   Thanks.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011401782


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoSerdeUtils.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.FunctionWithException;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** A util class with some helper method for serde in the MongoDB source. */
+@Internal
+public class MongoSerdeUtils {

Review Comment:
   Unit test added.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011667019


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSourceBuilder.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema;
+
+import org.bson.BsonDocument;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.flink.util.CollectionUtil.isNullOrEmpty;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The builder class for {@link MongoSource} to make it easier for the users to construct a {@link
+ * MongoSource}.
+ *
+ * @param <OUT> The output type of the source.
+ */
+@PublicEvolving
+public class MongoSourceBuilder<OUT> {
+
+    private final MongoConnectionOptions.MongoConnectionOptionsBuilder connectionOptionsBuilder;
+    private final MongoReadOptions.MongoReadOptionsBuilder readOptionsBuilder;
+
+    private List<String> projectedFields;
+    private int limit = -1;
+    private MongoDeserializationSchema<OUT> deserializationSchema;
+
+    MongoSourceBuilder() {
+        this.connectionOptionsBuilder = MongoConnectionOptions.builder();
+        this.readOptionsBuilder = MongoReadOptions.builder();
+    }
+
+    /**
+     * Sets the connection string of MongoDB.
+     *
+     * @param uri connection string of MongoDB
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setUri(String uri) {
+        connectionOptionsBuilder.setUri(uri);
+        return this;
+    }
+
+    /**
+     * Sets the database to sink of MongoDB.
+     *
+     * @param database the database to sink of MongoDB.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setDatabase(String database) {
+        connectionOptionsBuilder.setDatabase(database);
+        return this;
+    }
+
+    /**
+     * Sets the collection to sink of MongoDB.
+     *
+     * @param collection the collection to sink of MongoDB.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setCollection(String collection) {
+        connectionOptionsBuilder.setCollection(collection);
+        return this;
+    }
+
+    /**
+     * Sets the number of documents should be fetched per round-trip when reading.
+     *
+     * @param fetchSize the number of documents should be fetched per round-trip when reading.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setFetchSize(int fetchSize) {
+        readOptionsBuilder.setFetchSize(fetchSize);
+        return this;
+    }
+
+    /**
+     * Sets the batch size of MongoDB find cursor.
+     *
+     * @param cursorBatchSize the max batch size of find cursor.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setCursorBatchSize(int cursorBatchSize) {
+        readOptionsBuilder.setCursorBatchSize(cursorBatchSize);
+        return this;
+    }
+
+    /**
+     * Set this option to true to prevent cursor timeout (defaults to 10 minutes).
+     *
+     * @param noCursorTimeout Set this option to true to prevent cursor timeout.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setNoCursorTimeout(boolean noCursorTimeout) {
+        readOptionsBuilder.setNoCursorTimeout(noCursorTimeout);
+        return this;
+    }
+
+    /**
+     * Sets the partition strategy.
+     *
+     * @param partitionStrategy the strategy of a partition.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setPartitionStrategy(PartitionStrategy partitionStrategy) {
+        readOptionsBuilder.setPartitionStrategy(partitionStrategy);
+        return this;
+    }
+
+    /**
+     * Sets the partition size of MongoDB split.
+     *
+     * @param partitionSize the memory size of a partition.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setPartitionSize(MemorySize partitionSize) {
+        readOptionsBuilder.setPartitionSize(partitionSize);
+        return this;
+    }
+
+    /**
+     * Sets the samples size per partition only effective for sample partition strategy.
+     *
+     * @param samplesPerPartition the samples size per partition
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setSamplesPerPartition(int samplesPerPartition) {
+        readOptionsBuilder.setSamplesPerPartition(samplesPerPartition);
+        return this;
+    }
+
+    /**
+     * Sets the limit of documents to read.

Review Comment:
   What does it mean to not push the limit to MongoDB?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1011440345


##########
flink-connector-mongodb/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.architecture;
+
+import org.apache.flink.architecture.common.ImportOptions;
+
+import com.tngtech.archunit.core.importer.ImportOption;
+import com.tngtech.archunit.junit.AnalyzeClasses;
+import com.tngtech.archunit.junit.ArchTest;
+import com.tngtech.archunit.junit.ArchTests;
+
+/** Architecture tests for test code. */
+@AnalyzeClasses(
+        packages = "org.apache.flink.connector.mongodb",
+        importOptions = {
+            ImportOption.OnlyIncludeTests.class,
+            ImportOptions.ExcludeScalaImportOption.class,
+            ImportOptions.ExcludeShadedImportOption.class
+        })
+public class TestCodeArchitectureTest {
+
+    @ArchTest
+    public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class);
+}

Review Comment:
   It seems that there are currently no proper rules for production code.
   I only found `ITCaseRules` at `org.apache.flink.architecture.rules` package.
   Do we have rules that can be reused, or do we need to customize some of them?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010606545


##########
flink-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,165 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0.0-SNAPSHOT</version>
+		<relativePath>../pom.xml</relativePath>
+	</parent>
+
+	<artifactId>flink-connector-mongodb</artifactId>
+	<name>Flink : Connectors : MongoDB</name>
+
+	<packaging>jar</packaging>
+
+	<dependencies>
+		<!-- Core -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-base</artifactId>
+			<version>${flink.version}</version>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${flink.version}</version>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- MongoDB -->
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+		</dependency>
+
+		<!-- Table ecosystem -->
+
+		<!-- Projects depending on this project won't depend on flink-table-*. -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-api-java-bridge</artifactId>
+			<version>${flink.version}</version>
+			<scope>provided</scope>
+			<optional>true</optional>
+		</dependency>
+
+		<!-- Tests -->
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-runtime</artifactId>
+			<version>${flink.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-streaming-java</artifactId>
+			<version>${flink.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Table API integration tests -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-planner-loader</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-runtime</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-table-common</artifactId>
+			<version>${flink.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-base</artifactId>
+			<version>${flink.version}</version>
+			<type>test-jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-jar-plugin</artifactId>
+				<executions>
+					<execution>
+						<goals>
+							<goal>test-jar</goal>
+						</goals>
+					</execution>
+				</executions>
+			</plugin>
+		</plugins>
+	</build>

Review Comment:
   In order to give some reuse to the e2e test, it seems like it can be removed for now.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1010734402


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoSerdeUtils.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.FunctionWithException;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/** A util class with some helper method for serde in the MongoDB source. */
+@Internal
+public class MongoSerdeUtils {

Review Comment:
   Thanks, I'll add some test cases for this class.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1044077023


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,677 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = new Object[in.size()];
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {

Review Comment:
   Thanks, will fix it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1044075759


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSink.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.MongoSink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataSerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A {@link DynamicTableSink} for MongoDB. */
+@Internal
+public class MongoDynamicTableSink implements DynamicTableSink {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    @Nullable private final Integer parallelism;
+    private final DataType physicalRowDataType;
+    private final SerializableFunction<RowData, BsonValue> keyExtractor;
+
+    public MongoDynamicTableSink(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            @Nullable Integer parallelism,
+            DataType physicalRowDataType,
+            SerializableFunction<RowData, BsonValue> keyExtractor) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.parallelism = parallelism;
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.keyExtractor = checkNotNull(keyExtractor);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        return ChangelogMode.upsert();

Review Comment:
   Yes, changelog mode depend on the primary key.
   If user explicitly declares a primary key in flink sql, we use `MongoKeyExtractor` to extract them as `_id` of MongoDB. If not, we use `AppendOnlyKeyExtractor` to pass MongoDB a null `_id` and let MongoDB automatically generate ObjectId.
   
   Considering that there will be some situations where the primary key definition is forgotten, we may need to provide some configuration to explicitly specify whether to write as `append-only` or `upsert` mode so that we can do some verification at startup.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1044175579


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,677 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());

Review Comment:
   Thanks @twalthr to catch this.
   Both `RowDataToBsonConverters` and `BsonToRowDataConverters` are fixex.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1032654791


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitState.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/** MongoDB source split state. */
+@PublicEvolving
+public class MongoSourceSplitState {

Review Comment:
   This is designed to save extra states(in future), such as the current offset of change stream.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1030606725


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+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.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test for {@link MongoDynamicTableSource} and {@link MongoDynamicTableSink} created by {@link
+ * MongoDynamicTableFactory}.
+ */
+public class MongoDynamicTableFactoryTest {
+
+    private static final ResolvedSchema SCHEMA =
+            new ResolvedSchema(
+                    Arrays.asList(
+                            Column.physical("aaa", DataTypes.INT().notNull()),
+                            Column.physical("bbb", DataTypes.STRING().notNull()),
+                            Column.physical("ccc", DataTypes.DOUBLE()),
+                            Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
+                            Column.physical("eee", DataTypes.TIMESTAMP(3))),
+                    Collections.emptyList(),
+                    UniqueConstraint.primaryKey("name", Arrays.asList("bbb", "aaa")));
+
+    @Test
+    public void testMongoCommonProperties() {
+        Map<String, String> properties = getRequiredOptions();
+
+        // validation for source
+        DynamicTableSource actualSource = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expectedSource =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+        assertThat(actualSource).isEqualTo(expectedSource);
+
+        // validation for sink
+        DynamicTableSink actualSink = createTableSink(SCHEMA, properties);
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+        MongoDynamicTableSink expectedSink =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+        assertThat(actualSink).isEqualTo(expectedSink);
+    }
+
+    @Test
+    public void testMongoReadProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "1024");
+        properties.put("scan.cursor.batch-size", "2048");

Review Comment:
   Let's reference the actual `MongoConnectorOptions` whenever possible. (`ConfigOption#key`)



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.source.enumerator.MongoSourceEnumState;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoSourceSplit}. */
+@Internal
+public interface MongoSplitAssigner extends Serializable {
+
+    /**
+     * Called to open the assigner to acquire any resources, like threads or network connections.
+     */
+    void open();
+
+    /**
+     * Called to close the assigner, in case it holds on to any resources, like threads or network
+     * connections.
+     */
+    void close() throws IOException;
+
+    /** Gets the next split. */
+    Optional<MongoSourceSplit> getNext();
+
+    /**
+     * Adds a set of splits to this assigner. This happens for example when some split processing
+     * failed and the splits need to be re-added.
+     */
+    void addSplitsBack(Collection<MongoSourceSplit> splits);
+
+    /** Snapshot the current assign state into checkpoint. */
+    MongoSourceEnumState snapshotState(long checkpointId);
+
+    /** Return if there are no more splits. */

Review Comment:
   ```suggestion
       /** Return whether there are no more splits. */
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.source.enumerator.MongoSourceEnumState;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoSourceSplit}. */
+@Internal
+public interface MongoSplitAssigner extends Serializable {
+
+    /**
+     * Called to open the assigner to acquire any resources, like threads or network connections.
+     */
+    void open();
+
+    /**
+     * Called to close the assigner, in case it holds on to any resources, like threads or network
+     * connections.
+     */
+    void close() throws IOException;
+
+    /** Gets the next split. */
+    Optional<MongoSourceSplit> getNext();

Review Comment:
   When should this return an empty optional?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/assigner/MongoSplitAssigner.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.source.enumerator.MongoSourceEnumState;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.Optional;
+
+/** The split assigner for {@link MongoSourceSplit}. */
+@Internal
+public interface MongoSplitAssigner extends Serializable {

Review Comment:
   Does not need to be serializable afaict.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.time.Duration;
+
+/**
+ * Base options for the MongoDB connector. Needs to be public so that the {@link
+ * org.apache.flink.table.api.TableDescriptor} can access it.
+ */
+@PublicEvolving
+public class MongoConnectorOptions {
+
+    private MongoConnectorOptions() {}
+
+    public static final ConfigOption<String> URI =
+            ConfigOptions.key("uri")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the connection uri of MongoDB.");
+
+    public static final ConfigOption<String> DATABASE =
+            ConfigOptions.key("database")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the database to read or write of MongoDB.");
+
+    public static final ConfigOption<String> COLLECTION =
+            ConfigOptions.key("collection")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the collection to read or write of MongoDB.");
+
+    public static final ConfigOption<Integer> SCAN_FETCH_SIZE =
+            ConfigOptions.key("scan.fetch-size")
+                    .intType()
+                    .defaultValue(2048)
+                    .withDescription(
+                            "Gives the reader a hint as to the number of documents that should be fetched from the database per round-trip when reading. ");
+
+    public static final ConfigOption<Integer> SCAN_CURSOR_BATCH_SIZE =

Review Comment:
   I'm struggling to understand the difference between this option and `SCAN_FETCH_SIZE`. The description are very similar.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.time.Duration;
+
+/**
+ * Base options for the MongoDB connector. Needs to be public so that the {@link
+ * org.apache.flink.table.api.TableDescriptor} can access it.
+ */
+@PublicEvolving
+public class MongoConnectorOptions {
+
+    private MongoConnectorOptions() {}
+
+    public static final ConfigOption<String> URI =
+            ConfigOptions.key("uri")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the connection uri of MongoDB.");
+
+    public static final ConfigOption<String> DATABASE =
+            ConfigOptions.key("database")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the database to read or write of MongoDB.");
+
+    public static final ConfigOption<String> COLLECTION =
+            ConfigOptions.key("collection")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the collection to read or write of MongoDB.");
+
+    public static final ConfigOption<Integer> SCAN_FETCH_SIZE =
+            ConfigOptions.key("scan.fetch-size")
+                    .intType()
+                    .defaultValue(2048)
+                    .withDescription(
+                            "Gives the reader a hint as to the number of documents that should be fetched from the database per round-trip when reading. ");
+
+    public static final ConfigOption<Integer> SCAN_CURSOR_BATCH_SIZE =
+            ConfigOptions.key("scan.cursor.batch-size")
+                    .intType()
+                    .defaultValue(0)
+                    .withDescription(
+                            "Specifies the number of documents to return in each batch of the response from the MongoDB instance. Set to 0 to use server's default.");
+
+    public static final ConfigOption<Boolean> SCAN_CURSOR_NO_TIMEOUT =
+            ConfigOptions.key("scan.cursor.no-timeout")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription(
+                            "The server normally times out idle cursors after an inactivity"
+                                    + " period (10 minutes) to prevent excess memory use. Set this option to true to prevent that."
+                                    + " However, if the application takes longer than 30 minutes to process the current batch of documents,"
+                                    + " the session is marked as expired and closed.");
+
+    public static final ConfigOption<PartitionStrategy> SCAN_PARTITION_STRATEGY =
+            ConfigOptions.key("scan.partition.strategy")
+                    .enumType(PartitionStrategy.class)
+                    .defaultValue(PartitionStrategy.DEFAULT)
+                    .withDescription(
+                            "Specifies the partition strategy. Available strategies are single, sample, split-vector, sharded and default.");
+
+    public static final ConfigOption<MemorySize> SCAN_PARTITION_SIZE =
+            ConfigOptions.key("scan.partition.size")
+                    .memoryType()
+                    .defaultValue(MemorySize.parse("64mb"))
+                    .withDescription("Specifies the partition memory size.");
+
+    public static final ConfigOption<Integer> SCAN_PARTITION_SAMPLES =
+            ConfigOptions.key("scan.partition.samples")
+                    .intType()
+                    .defaultValue(10)
+                    .withDescription(
+                            "Specifies the the samples count per partition. It only takes effect when the partition strategy is sample. "
+                                    + "The sample partitioner samples the collection, projects and sorts by the partition fields. "
+                                    + "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<Duration> LOOKUP_RETRY_INTERVAL =
+            ConfigOptions.key("lookup.retry.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofMillis(1000L))
+                    .withDescription(
+                            "Specifies the retry time interval if lookup records from database failed.");
+
+    public static final ConfigOption<Integer> BUFFER_FLUSH_MAX_ROWS =
+            ConfigOptions.key("sink.buffer-flush.max-rows")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "Specifies the maximum number of buffered rows per bulk request.");
+
+    public static final ConfigOption<Duration> BUFFER_FLUSH_INTERVAL =
+            ConfigOptions.key("sink.buffer-flush.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofSeconds(1))
+                    .withDescription("Specifies the bulk flush interval.");

Review Comment:
   ```suggestion
                       .withDescription("Specifies the batch flush interval.");
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+    }
+
+    @Test
+    public void testShardedSplitter() {
+        MongoNamespace namespace = new MongoNamespace("test_db.test_coll");
+        BsonDocument mockCollectionMetadata = mockCollectionMetadata();
+        List<BsonDocument> mockChunksData = mockChunksData();
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(
+                        MongoReadOptions.builder().build(),
+                        mongoClient,
+                        namespace,
+                        mockCollStats());
+
+        List<MongoScanSourceSplit> expected = new ArrayList<>();
+        for (int i = 0; i < mockChunksData.size(); i++) {
+            BsonDocument mockChunkData = mockChunksData.get(i);
+            expected.add(
+                    new MongoScanSourceSplit(
+                            String.format("%s_%d", namespace, i),
+                            namespace.getDatabaseName(),
+                            namespace.getCollectionName(),
+                            mockChunkData.getDocument(MIN_FIELD),
+                            mockChunkData.getDocument(MAX_FIELD),
+                            mockCollectionMetadata.getDocument(KEY_FIELD)));
+        }
+
+        try (MockedStatic<MongoUtils> util = mockStatic(MongoUtils.class)) {
+            util.when(() -> MongoUtils.readCollectionMetadata(any(), any()))
+                    .thenReturn(Optional.of(mockCollectionMetadata));
+
+            util.when(() -> MongoUtils.readChunks(any(), any())).thenReturn(mockChunksData);
+
+            util.when(() -> MongoUtils.isShardedCollectionDropped(any())).thenReturn(false);
+
+            Collection<MongoScanSourceSplit> actual =
+                    MongoShardedSplitter.INSTANCE.split(splitContext);
+            assertThat(actual).isEqualTo(expected);
+        }
+    }
+
+    private BsonDocument mockCollectionMetadata() {

Review Comment:
   ```suggestion
       private static BsonDocument mockCollectionMetadata() {
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+    }
+
+    @Test
+    public void testShardedSplitter() {
+        MongoNamespace namespace = new MongoNamespace("test_db.test_coll");
+        BsonDocument mockCollectionMetadata = mockCollectionMetadata();
+        List<BsonDocument> mockChunksData = mockChunksData();
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(
+                        MongoReadOptions.builder().build(),
+                        mongoClient,
+                        namespace,
+                        mockCollStats());
+
+        List<MongoScanSourceSplit> expected = new ArrayList<>();
+        for (int i = 0; i < mockChunksData.size(); i++) {
+            BsonDocument mockChunkData = mockChunksData.get(i);
+            expected.add(
+                    new MongoScanSourceSplit(
+                            String.format("%s_%d", namespace, i),
+                            namespace.getDatabaseName(),
+                            namespace.getCollectionName(),
+                            mockChunkData.getDocument(MIN_FIELD),
+                            mockChunkData.getDocument(MAX_FIELD),
+                            mockCollectionMetadata.getDocument(KEY_FIELD)));
+        }
+
+        try (MockedStatic<MongoUtils> util = mockStatic(MongoUtils.class)) {
+            util.when(() -> MongoUtils.readCollectionMetadata(any(), any()))
+                    .thenReturn(Optional.of(mockCollectionMetadata));
+
+            util.when(() -> MongoUtils.readChunks(any(), any())).thenReturn(mockChunksData);
+
+            util.when(() -> MongoUtils.isShardedCollectionDropped(any())).thenReturn(false);
+
+            Collection<MongoScanSourceSplit> actual =
+                    MongoShardedSplitter.INSTANCE.split(splitContext);
+            assertThat(actual).isEqualTo(expected);
+        }
+    }
+
+    private BsonDocument mockCollectionMetadata() {
+        return new BsonDocument()
+                .append(ID_FIELD, new BsonObjectId())
+                .append(UUID_FIELD, new BsonBinary(UUID.randomUUID()))
+                .append(DROPPED_FIELD, BsonBoolean.FALSE)
+                .append(KEY_FIELD, ID_HINT);
+    }
+
+    private List<BsonDocument> mockChunksData() {
+        return Arrays.asList(mockChunkData(1), mockChunkData(2), mockChunkData(3));
+    }
+
+    private BsonDocument mockChunkData(int index) {

Review Comment:
   ```suggestion
       private static BsonDocument mockChunkData(int index) {
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+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.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test for {@link MongoDynamicTableSource} and {@link MongoDynamicTableSink} created by {@link
+ * MongoDynamicTableFactory}.
+ */
+public class MongoDynamicTableFactoryTest {
+
+    private static final ResolvedSchema SCHEMA =
+            new ResolvedSchema(
+                    Arrays.asList(
+                            Column.physical("aaa", DataTypes.INT().notNull()),
+                            Column.physical("bbb", DataTypes.STRING().notNull()),
+                            Column.physical("ccc", DataTypes.DOUBLE()),
+                            Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
+                            Column.physical("eee", DataTypes.TIMESTAMP(3))),
+                    Collections.emptyList(),
+                    UniqueConstraint.primaryKey("name", Arrays.asList("bbb", "aaa")));
+
+    @Test
+    public void testMongoCommonProperties() {
+        Map<String, String> properties = getRequiredOptions();
+
+        // validation for source
+        DynamicTableSource actualSource = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expectedSource =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+        assertThat(actualSource).isEqualTo(expectedSource);
+
+        // validation for sink
+        DynamicTableSink actualSink = createTableSink(SCHEMA, properties);
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+        MongoDynamicTableSink expectedSink =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+        assertThat(actualSink).isEqualTo(expectedSink);
+    }
+
+    @Test
+    public void testMongoReadProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "1024");
+        properties.put("scan.cursor.batch-size", "2048");
+        properties.put("scan.cursor.no-timeout", "false");
+        properties.put("scan.partition.strategy", "split-vector");
+        properties.put("scan.partition.size", "128m");
+        properties.put("scan.partition.samples", "5");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoReadOptions readOptions =
+                MongoReadOptions.builder()
+                        .setFetchSize(1024)
+                        .setCursorBatchSize(2048)
+                        .setNoCursorTimeout(false)
+                        .setPartitionStrategy(PartitionStrategy.SPLIT_VECTOR)
+                        .setPartitionSize(MemorySize.ofMebiBytes(128))
+                        .setSamplesPerPartition(5)
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        readOptions,
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoLookupProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("lookup.cache", "PARTIAL");
+        properties.put("lookup.partial-cache.expire-after-write", "10s");
+        properties.put("lookup.partial-cache.expire-after-access", "20s");
+        properties.put("lookup.partial-cache.cache-missing-key", "false");
+        properties.put("lookup.partial-cache.max-rows", "15213");
+        properties.put("lookup.max-retries", "10");
+        properties.put("lookup.retry.interval", "20ms");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        DefaultLookupCache.fromConfig(Configuration.fromMap(properties)),
+                        10,
+                        20,
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.buffer-flush.max-rows", "1001");
+        properties.put("sink.buffer-flush.interval", "2min");
+        properties.put("sink.delivery-guarantee", "at-least-once");
+        properties.put("sink.max-retries", "5");
+        properties.put("sink.retry.interval", "2s");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoWriteOptions writeOptions =
+                MongoWriteOptions.builder()
+                        .setBatchSize(1001)
+                        .setBatchIntervalMs(TimeUnit.MINUTES.toMillis(2))
+                        .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+                        .setMaxRetries(5)
+                        .setRetryIntervalMs(TimeUnit.SECONDS.toMillis(2))
+                        .build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkWithParallelism() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.parallelism", "2");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        2,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoValidation() {
+        // fetch size lower than 1
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "0");
+
+        Map<String, String> finalProperties1 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties1))
+                .hasStackTraceContaining("The fetch size must be larger than 0.");
+
+        // cursor batch size lower than 0
+        properties = getRequiredOptions();
+        properties.put("scan.cursor.batch-size", "-1");
+
+        Map<String, String> finalProperties2 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties2))
+                .hasStackTraceContaining("The cursor batch size must be larger than or equal to 0");
+
+        // partition memory size lower than 1mb
+        properties = getRequiredOptions();
+        properties.put("scan.partition.size", "900kb");
+
+        Map<String, String> finalProperties3 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties3))
+                .hasStackTraceContaining(
+                        "The partition size must be larger than or equals to 1mb.");
+
+        // samples per partition lower than 1
+        properties = getRequiredOptions();
+        properties.put("scan.partition.samples", "0");
+
+        Map<String, String> finalProperties4 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties4))
+                .hasStackTraceContaining("The samples per partition must be larger than 0.");
+
+        // lookup retry times shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("lookup.max-retries", "-1");
+        Map<String, String> finalProperties5 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties5))
+                .hasStackTraceContaining(
+                        "The 'lookup.max-retries' must be larger than or equals to 0.");
+
+        // lookup retry interval shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("lookup.retry.interval", "0ms");
+        Map<String, String> finalProperties6 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties6))
+                .hasStackTraceContaining("The 'lookup.retry.interval' must be larger than 0.");
+
+        // sink retries shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("sink.max-retries", "-1");
+        Map<String, String> finalProperties7 = properties;
+        assertThatThrownBy(() -> createTableSink(SCHEMA, finalProperties7))
+                .hasStackTraceContaining(
+                        "The sink max retry times must be larger than or equal to 0.");
+
+        // sink retry interval shouldn't be negative

Review Comment:
   ```suggestion
           // sink retry interval shouldn't be 0
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+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.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test for {@link MongoDynamicTableSource} and {@link MongoDynamicTableSink} created by {@link
+ * MongoDynamicTableFactory}.
+ */
+public class MongoDynamicTableFactoryTest {
+
+    private static final ResolvedSchema SCHEMA =
+            new ResolvedSchema(
+                    Arrays.asList(
+                            Column.physical("aaa", DataTypes.INT().notNull()),
+                            Column.physical("bbb", DataTypes.STRING().notNull()),
+                            Column.physical("ccc", DataTypes.DOUBLE()),
+                            Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
+                            Column.physical("eee", DataTypes.TIMESTAMP(3))),
+                    Collections.emptyList(),
+                    UniqueConstraint.primaryKey("name", Arrays.asList("bbb", "aaa")));
+
+    @Test
+    public void testMongoCommonProperties() {
+        Map<String, String> properties = getRequiredOptions();
+
+        // validation for source
+        DynamicTableSource actualSource = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expectedSource =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+        assertThat(actualSource).isEqualTo(expectedSource);
+
+        // validation for sink
+        DynamicTableSink actualSink = createTableSink(SCHEMA, properties);
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+        MongoDynamicTableSink expectedSink =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+        assertThat(actualSink).isEqualTo(expectedSink);
+    }
+
+    @Test
+    public void testMongoReadProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "1024");
+        properties.put("scan.cursor.batch-size", "2048");
+        properties.put("scan.cursor.no-timeout", "false");
+        properties.put("scan.partition.strategy", "split-vector");
+        properties.put("scan.partition.size", "128m");
+        properties.put("scan.partition.samples", "5");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoReadOptions readOptions =
+                MongoReadOptions.builder()
+                        .setFetchSize(1024)
+                        .setCursorBatchSize(2048)
+                        .setNoCursorTimeout(false)
+                        .setPartitionStrategy(PartitionStrategy.SPLIT_VECTOR)
+                        .setPartitionSize(MemorySize.ofMebiBytes(128))
+                        .setSamplesPerPartition(5)
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        readOptions,
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoLookupProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("lookup.cache", "PARTIAL");
+        properties.put("lookup.partial-cache.expire-after-write", "10s");
+        properties.put("lookup.partial-cache.expire-after-access", "20s");
+        properties.put("lookup.partial-cache.cache-missing-key", "false");
+        properties.put("lookup.partial-cache.max-rows", "15213");
+        properties.put("lookup.max-retries", "10");
+        properties.put("lookup.retry.interval", "20ms");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        DefaultLookupCache.fromConfig(Configuration.fromMap(properties)),
+                        10,
+                        20,
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.buffer-flush.max-rows", "1001");
+        properties.put("sink.buffer-flush.interval", "2min");
+        properties.put("sink.delivery-guarantee", "at-least-once");
+        properties.put("sink.max-retries", "5");
+        properties.put("sink.retry.interval", "2s");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoWriteOptions writeOptions =
+                MongoWriteOptions.builder()
+                        .setBatchSize(1001)
+                        .setBatchIntervalMs(TimeUnit.MINUTES.toMillis(2))
+                        .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+                        .setMaxRetries(5)
+                        .setRetryIntervalMs(TimeUnit.SECONDS.toMillis(2))
+                        .build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkWithParallelism() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.parallelism", "2");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        2,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoValidation() {
+        // fetch size lower than 1
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "0");
+
+        Map<String, String> finalProperties1 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties1))
+                .hasStackTraceContaining("The fetch size must be larger than 0.");
+
+        // cursor batch size lower than 0
+        properties = getRequiredOptions();
+        properties.put("scan.cursor.batch-size", "-1");
+
+        Map<String, String> finalProperties2 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties2))
+                .hasStackTraceContaining("The cursor batch size must be larger than or equal to 0");
+
+        // partition memory size lower than 1mb
+        properties = getRequiredOptions();
+        properties.put("scan.partition.size", "900kb");
+
+        Map<String, String> finalProperties3 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties3))
+                .hasStackTraceContaining(
+                        "The partition size must be larger than or equals to 1mb.");
+
+        // samples per partition lower than 1
+        properties = getRequiredOptions();
+        properties.put("scan.partition.samples", "0");
+
+        Map<String, String> finalProperties4 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties4))
+                .hasStackTraceContaining("The samples per partition must be larger than 0.");
+
+        // lookup retry times shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("lookup.max-retries", "-1");
+        Map<String, String> finalProperties5 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties5))
+                .hasStackTraceContaining(
+                        "The 'lookup.max-retries' must be larger than or equals to 0.");
+
+        // lookup retry interval shouldn't be negative

Review Comment:
   ```suggestion
           // lookup retry interval shouldn't be 0
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+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.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test for {@link MongoDynamicTableSource} and {@link MongoDynamicTableSink} created by {@link
+ * MongoDynamicTableFactory}.
+ */
+public class MongoDynamicTableFactoryTest {
+
+    private static final ResolvedSchema SCHEMA =
+            new ResolvedSchema(
+                    Arrays.asList(
+                            Column.physical("aaa", DataTypes.INT().notNull()),
+                            Column.physical("bbb", DataTypes.STRING().notNull()),
+                            Column.physical("ccc", DataTypes.DOUBLE()),
+                            Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
+                            Column.physical("eee", DataTypes.TIMESTAMP(3))),
+                    Collections.emptyList(),
+                    UniqueConstraint.primaryKey("name", Arrays.asList("bbb", "aaa")));
+
+    @Test
+    public void testMongoCommonProperties() {
+        Map<String, String> properties = getRequiredOptions();
+
+        // validation for source
+        DynamicTableSource actualSource = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expectedSource =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+        assertThat(actualSource).isEqualTo(expectedSource);
+
+        // validation for sink
+        DynamicTableSink actualSink = createTableSink(SCHEMA, properties);
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+        MongoDynamicTableSink expectedSink =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+        assertThat(actualSink).isEqualTo(expectedSink);
+    }
+
+    @Test
+    public void testMongoReadProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "1024");
+        properties.put("scan.cursor.batch-size", "2048");
+        properties.put("scan.cursor.no-timeout", "false");
+        properties.put("scan.partition.strategy", "split-vector");
+        properties.put("scan.partition.size", "128m");
+        properties.put("scan.partition.samples", "5");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoReadOptions readOptions =
+                MongoReadOptions.builder()
+                        .setFetchSize(1024)
+                        .setCursorBatchSize(2048)
+                        .setNoCursorTimeout(false)
+                        .setPartitionStrategy(PartitionStrategy.SPLIT_VECTOR)
+                        .setPartitionSize(MemorySize.ofMebiBytes(128))
+                        .setSamplesPerPartition(5)
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        readOptions,
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoLookupProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("lookup.cache", "PARTIAL");
+        properties.put("lookup.partial-cache.expire-after-write", "10s");
+        properties.put("lookup.partial-cache.expire-after-access", "20s");
+        properties.put("lookup.partial-cache.cache-missing-key", "false");
+        properties.put("lookup.partial-cache.max-rows", "15213");
+        properties.put("lookup.max-retries", "10");
+        properties.put("lookup.retry.interval", "20ms");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        DefaultLookupCache.fromConfig(Configuration.fromMap(properties)),
+                        10,
+                        20,
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.buffer-flush.max-rows", "1001");
+        properties.put("sink.buffer-flush.interval", "2min");
+        properties.put("sink.delivery-guarantee", "at-least-once");
+        properties.put("sink.max-retries", "5");
+        properties.put("sink.retry.interval", "2s");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoWriteOptions writeOptions =
+                MongoWriteOptions.builder()
+                        .setBatchSize(1001)
+                        .setBatchIntervalMs(TimeUnit.MINUTES.toMillis(2))
+                        .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+                        .setMaxRetries(5)
+                        .setRetryIntervalMs(TimeUnit.SECONDS.toMillis(2))
+                        .build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkWithParallelism() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.parallelism", "2");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        2,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoValidation() {
+        // fetch size lower than 1
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "0");
+
+        Map<String, String> finalProperties1 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties1))
+                .hasStackTraceContaining("The fetch size must be larger than 0.");
+
+        // cursor batch size lower than 0
+        properties = getRequiredOptions();
+        properties.put("scan.cursor.batch-size", "-1");
+
+        Map<String, String> finalProperties2 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties2))
+                .hasStackTraceContaining("The cursor batch size must be larger than or equal to 0");
+
+        // partition memory size lower than 1mb
+        properties = getRequiredOptions();
+        properties.put("scan.partition.size", "900kb");
+
+        Map<String, String> finalProperties3 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties3))
+                .hasStackTraceContaining(
+                        "The partition size must be larger than or equals to 1mb.");
+
+        // samples per partition lower than 1
+        properties = getRequiredOptions();
+        properties.put("scan.partition.samples", "0");
+
+        Map<String, String> finalProperties4 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties4))
+                .hasStackTraceContaining("The samples per partition must be larger than 0.");
+
+        // lookup retry times shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("lookup.max-retries", "-1");
+        Map<String, String> finalProperties5 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties5))
+                .hasStackTraceContaining(
+                        "The 'lookup.max-retries' must be larger than or equals to 0.");
+
+        // lookup retry interval shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("lookup.retry.interval", "0ms");
+        Map<String, String> finalProperties6 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties6))
+                .hasStackTraceContaining("The 'lookup.retry.interval' must be larger than 0.");
+
+        // sink retries shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("sink.max-retries", "-1");
+        Map<String, String> finalProperties7 = properties;
+        assertThatThrownBy(() -> createTableSink(SCHEMA, finalProperties7))
+                .hasStackTraceContaining(
+                        "The sink max retry times must be larger than or equal to 0.");
+
+        // sink retry interval shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("sink.retry.interval", "0ms");
+        Map<String, String> finalProperties8 = properties;
+        assertThatThrownBy(() -> createTableSink(SCHEMA, finalProperties8))
+                .hasStackTraceContaining(
+                        "The retry interval (in milliseconds) must be larger than 0.");
+
+        // sink buffered rows shouldn't be smaller than 1
+        properties = getRequiredOptions();
+        properties.put("sink.buffer-flush.max-rows", "0");
+        Map<String, String> finalProperties9 = properties;
+        assertThatThrownBy(() -> createTableSink(SCHEMA, finalProperties9))
+                .hasStackTraceContaining("Max number of batch size must be larger than 0.");
+    }
+
+    private Map<String, String> getRequiredOptions() {

Review Comment:
   ```suggestion
       private static Map<String, String> getRequiredOptions() {
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.table.converter.BsonToRowDataConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.LookupFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+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;
+import org.bson.conversions.Bson;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+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.project;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A lookup function for {@link MongoDynamicTableSource}. */
+@Internal
+public class MongoRowDataLookupFunction extends LookupFunction {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoRowDataLookupFunction.class);
+    private static final long serialVersionUID = 1L;
+
+    private final MongoConnectionOptions connectionOptions;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+
+    private final List<String> fieldNames;
+    private final List<String> keyNames;
+
+    private final BsonToRowDataConverters.BsonToRowDataConverter mongoRowConverter;
+    private final RowDataToBsonConverters.RowDataToBsonConverter lookupKeyRowConverter;
+
+    private transient MongoClient mongoClient;
+
+    public MongoRowDataLookupFunction(
+            MongoConnectionOptions connectionOptions,
+            int maxRetries,
+            long retryIntervalMs,
+            List<String> fieldNames,
+            List<DataType> fieldTypes,
+            List<String> keyNames,
+            RowType rowType) {
+        checkNotNull(fieldNames, "No fieldNames supplied.");
+        checkNotNull(fieldTypes, "No fieldTypes supplied.");
+        checkNotNull(keyNames, "No keyNames supplied.");
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.fieldNames = fieldNames;
+        this.mongoRowConverter = BsonToRowDataConverters.createNullableConverter(rowType);
+
+        this.keyNames = keyNames;
+        LogicalType[] keyTypes =
+                this.keyNames.stream()
+                        .map(
+                                s -> {
+                                    checkArgument(
+                                            fieldNames.contains(s),
+                                            "keyName %s can't find in fieldNames %s.",
+                                            s,
+                                            fieldNames);
+                                    return fieldTypes.get(fieldNames.indexOf(s)).getLogicalType();
+                                })
+                        .toArray(LogicalType[]::new);
+
+        this.lookupKeyRowConverter =
+                RowDataToBsonConverters.createNullableConverter(
+                        RowType.of(keyTypes, keyNames.toArray(new String[0])));
+    }
+
+    @Override
+    public void open(FunctionContext context) {
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    /**
+     * This is a lookup method which is called by Flink framework in runtime.
+     *
+     * @param keyRow lookup keys
+     */
+    @Override
+    public Collection<RowData> lookup(RowData keyRow) {
+        for (int retry = 0; retry <= maxRetries; retry++) {
+            try {
+                BsonDocument lookupValues = (BsonDocument) lookupKeyRowConverter.convert(keyRow);
+
+                List<Bson> filters =
+                        keyNames.stream()
+                                .map(name -> eq(name, lookupValues.get(name)))
+                                .collect(Collectors.toList());
+                Bson query = and(filters);
+
+                Bson projection = project(fieldNames);
+
+                try (MongoCursor<BsonDocument> cursor =
+                        getMongoCollection().find(query).projection(projection).cursor()) {
+                    ArrayList<RowData> rows = new ArrayList<>();

Review Comment:
   ```suggestion
                       List<RowData> rows = new ArrayList<>();
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/split/MongoSourceSplitState.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/** MongoDB source split state. */
+@PublicEvolving
+public class MongoSourceSplitState {

Review Comment:
   What's the idea behind this class? Why couldn't we work directly against the `MongoSourceSplit`?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.table.converter.BsonToRowDataConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.LookupFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+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;
+import org.bson.conversions.Bson;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+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.project;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A lookup function for {@link MongoDynamicTableSource}. */
+@Internal
+public class MongoRowDataLookupFunction extends LookupFunction {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoRowDataLookupFunction.class);
+    private static final long serialVersionUID = 1L;
+
+    private final MongoConnectionOptions connectionOptions;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+
+    private final List<String> fieldNames;
+    private final List<String> keyNames;
+
+    private final BsonToRowDataConverters.BsonToRowDataConverter mongoRowConverter;
+    private final RowDataToBsonConverters.RowDataToBsonConverter lookupKeyRowConverter;
+
+    private transient MongoClient mongoClient;
+
+    public MongoRowDataLookupFunction(
+            MongoConnectionOptions connectionOptions,
+            int maxRetries,
+            long retryIntervalMs,
+            List<String> fieldNames,
+            List<DataType> fieldTypes,
+            List<String> keyNames,
+            RowType rowType) {
+        checkNotNull(fieldNames, "No fieldNames supplied.");
+        checkNotNull(fieldTypes, "No fieldTypes supplied.");
+        checkNotNull(keyNames, "No keyNames supplied.");
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.fieldNames = fieldNames;
+        this.mongoRowConverter = BsonToRowDataConverters.createNullableConverter(rowType);
+
+        this.keyNames = keyNames;
+        LogicalType[] keyTypes =
+                this.keyNames.stream()
+                        .map(
+                                s -> {
+                                    checkArgument(
+                                            fieldNames.contains(s),
+                                            "keyName %s can't find in fieldNames %s.",
+                                            s,
+                                            fieldNames);
+                                    return fieldTypes.get(fieldNames.indexOf(s)).getLogicalType();
+                                })
+                        .toArray(LogicalType[]::new);
+
+        this.lookupKeyRowConverter =
+                RowDataToBsonConverters.createNullableConverter(
+                        RowType.of(keyTypes, keyNames.toArray(new String[0])));
+    }
+
+    @Override
+    public void open(FunctionContext context) {
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    /**
+     * This is a lookup method which is called by Flink framework in runtime.
+     *
+     * @param keyRow lookup keys
+     */
+    @Override
+    public Collection<RowData> lookup(RowData keyRow) {
+        for (int retry = 0; retry <= maxRetries; retry++) {
+            try {
+                BsonDocument lookupValues = (BsonDocument) lookupKeyRowConverter.convert(keyRow);
+
+                List<Bson> filters =
+                        keyNames.stream()
+                                .map(name -> eq(name, lookupValues.get(name)))
+                                .collect(Collectors.toList());
+                Bson query = and(filters);
+
+                Bson projection = project(fieldNames);
+
+                try (MongoCursor<BsonDocument> cursor =
+                        getMongoCollection().find(query).projection(projection).cursor()) {
+                    ArrayList<RowData> rows = new ArrayList<>();
+                    while (cursor.hasNext()) {
+                        RowData row = (RowData) mongoRowConverter.convert(cursor.next());
+                        rows.add(row);
+                    }
+                    rows.trimToSize();

Review Comment:
   ```suggestion
   ```
   This is unnecessary overhead that just reduces performance, allocating another array that will also be thrown away immediately after the result of this method has been processed.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.table.converter.BsonToRowDataConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.LookupFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+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;
+import org.bson.conversions.Bson;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+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.project;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A lookup function for {@link MongoDynamicTableSource}. */
+@Internal
+public class MongoRowDataLookupFunction extends LookupFunction {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoRowDataLookupFunction.class);
+    private static final long serialVersionUID = 1L;
+
+    private final MongoConnectionOptions connectionOptions;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+
+    private final List<String> fieldNames;
+    private final List<String> keyNames;
+
+    private final BsonToRowDataConverters.BsonToRowDataConverter mongoRowConverter;
+    private final RowDataToBsonConverters.RowDataToBsonConverter lookupKeyRowConverter;
+
+    private transient MongoClient mongoClient;
+
+    public MongoRowDataLookupFunction(
+            MongoConnectionOptions connectionOptions,
+            int maxRetries,
+            long retryIntervalMs,
+            List<String> fieldNames,
+            List<DataType> fieldTypes,
+            List<String> keyNames,
+            RowType rowType) {
+        checkNotNull(fieldNames, "No fieldNames supplied.");
+        checkNotNull(fieldTypes, "No fieldTypes supplied.");
+        checkNotNull(keyNames, "No keyNames supplied.");
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.fieldNames = fieldNames;
+        this.mongoRowConverter = BsonToRowDataConverters.createNullableConverter(rowType);
+
+        this.keyNames = keyNames;
+        LogicalType[] keyTypes =
+                this.keyNames.stream()
+                        .map(
+                                s -> {
+                                    checkArgument(
+                                            fieldNames.contains(s),
+                                            "keyName %s can't find in fieldNames %s.",
+                                            s,
+                                            fieldNames);
+                                    return fieldTypes.get(fieldNames.indexOf(s)).getLogicalType();
+                                })
+                        .toArray(LogicalType[]::new);
+
+        this.lookupKeyRowConverter =
+                RowDataToBsonConverters.createNullableConverter(
+                        RowType.of(keyTypes, keyNames.toArray(new String[0])));
+    }
+
+    @Override
+    public void open(FunctionContext context) {
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    /**
+     * This is a lookup method which is called by Flink framework in runtime.
+     *
+     * @param keyRow lookup keys
+     */
+    @Override
+    public Collection<RowData> lookup(RowData keyRow) {
+        for (int retry = 0; retry <= maxRetries; retry++) {
+            try {
+                BsonDocument lookupValues = (BsonDocument) lookupKeyRowConverter.convert(keyRow);
+
+                List<Bson> filters =
+                        keyNames.stream()
+                                .map(name -> eq(name, lookupValues.get(name)))
+                                .collect(Collectors.toList());
+                Bson query = and(filters);
+
+                Bson projection = project(fieldNames);
+
+                try (MongoCursor<BsonDocument> cursor =
+                        getMongoCollection().find(query).projection(projection).cursor()) {
+                    ArrayList<RowData> rows = new ArrayList<>();
+                    while (cursor.hasNext()) {
+                        RowData row = (RowData) mongoRowConverter.convert(cursor.next());
+                        rows.add(row);
+                    }
+                    rows.trimToSize();
+                    return rows;
+                }
+            } catch (MongoException e) {
+                LOG.debug("MongoDB lookup error, retry times = {}", retry, e);
+                if (retry >= maxRetries) {
+                    LOG.error("MongoDB lookup error", e);
+                    throw new RuntimeException("Execution of MongoDB lookup failed.", e);
+                }
+                try {
+                    Thread.sleep(retryIntervalMs * (retry + 1));
+                } catch (InterruptedException e1) {
+                    throw new RuntimeException(e1);

Review Comment:
   ```suggestion
                   } catch (InterruptedException e1) {
                       Thread.currentThread().interrupt();
                       throw new RuntimeException(e1);
   ```
   We must restore the interrupt flag so that any blocking calls outside this method are also interrupted.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoSourceSplitReader.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.reader.split;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.base.source.reader.splitreader.SplitReader;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import org.bson.BsonDocument;
+
+/**
+ * An split reader implements {@link SplitReader} for {@link MongoSourceSplit}.
+ *
+ * @param <T> Mongo source split.
+ */
+@Internal
+public interface MongoSourceSplitReader<T extends MongoSourceSplit>

Review Comment:
   What values does this interface add? Doesn't seem like any code is actually working against it.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.time.Duration;
+
+/**
+ * Base options for the MongoDB connector. Needs to be public so that the {@link
+ * org.apache.flink.table.api.TableDescriptor} can access it.
+ */
+@PublicEvolving
+public class MongoConnectorOptions {
+
+    private MongoConnectorOptions() {}
+
+    public static final ConfigOption<String> URI =
+            ConfigOptions.key("uri")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the connection uri of MongoDB.");
+
+    public static final ConfigOption<String> DATABASE =
+            ConfigOptions.key("database")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the database to read or write of MongoDB.");
+
+    public static final ConfigOption<String> COLLECTION =
+            ConfigOptions.key("collection")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the collection to read or write of MongoDB.");
+
+    public static final ConfigOption<Integer> SCAN_FETCH_SIZE =
+            ConfigOptions.key("scan.fetch-size")
+                    .intType()
+                    .defaultValue(2048)
+                    .withDescription(
+                            "Gives the reader a hint as to the number of documents that should be fetched from the database per round-trip when reading. ");
+
+    public static final ConfigOption<Integer> SCAN_CURSOR_BATCH_SIZE =
+            ConfigOptions.key("scan.cursor.batch-size")
+                    .intType()
+                    .defaultValue(0)
+                    .withDescription(
+                            "Specifies the number of documents to return in each batch of the response from the MongoDB instance. Set to 0 to use server's default.");
+
+    public static final ConfigOption<Boolean> SCAN_CURSOR_NO_TIMEOUT =
+            ConfigOptions.key("scan.cursor.no-timeout")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription(
+                            "The server normally times out idle cursors after an inactivity"
+                                    + " period (10 minutes) to prevent excess memory use. Set this option to true to prevent that."
+                                    + " However, if the application takes longer than 30 minutes to process the current batch of documents,"
+                                    + " the session is marked as expired and closed.");
+
+    public static final ConfigOption<PartitionStrategy> SCAN_PARTITION_STRATEGY =
+            ConfigOptions.key("scan.partition.strategy")
+                    .enumType(PartitionStrategy.class)
+                    .defaultValue(PartitionStrategy.DEFAULT)
+                    .withDescription(
+                            "Specifies the partition strategy. Available strategies are single, sample, split-vector, sharded and default.");

Review Comment:
   needs some description on what the differences are.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactory.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.table.config.MongoConfiguration;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.HashSet;
+import java.util.Set;
+
+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.COLLECTION;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.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;
+
+/**
+ * Factory for creating configured instances of {@link MongoDynamicTableSource} and {@link
+ * MongoDynamicTableSink}.
+ */
+@Internal
+public class MongoDynamicTableFactory
+        implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+    public static final String IDENTIFIER = "mongodb";
+
+    @Override
+    public String factoryIdentifier() {
+        return IDENTIFIER;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> requiredOptions() {
+        final Set<ConfigOption<?>> requiredOptions = new HashSet<>();
+        requiredOptions.add(URI);
+        requiredOptions.add(DATABASE);
+        requiredOptions.add(COLLECTION);
+        return requiredOptions;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> optionalOptions() {
+        final Set<ConfigOption<?>> optionalOptions = new HashSet<>();
+        optionalOptions.add(SCAN_FETCH_SIZE);
+        optionalOptions.add(SCAN_CURSOR_BATCH_SIZE);
+        optionalOptions.add(SCAN_CURSOR_NO_TIMEOUT);
+        optionalOptions.add(SCAN_PARTITION_STRATEGY);
+        optionalOptions.add(SCAN_PARTITION_SIZE);
+        optionalOptions.add(SCAN_PARTITION_SAMPLES);
+        optionalOptions.add(BUFFER_FLUSH_MAX_ROWS);
+        optionalOptions.add(BUFFER_FLUSH_INTERVAL);
+        optionalOptions.add(DELIVERY_GUARANTEE);
+        optionalOptions.add(SINK_MAX_RETRIES);
+        optionalOptions.add(SINK_RETRY_INTERVAL);
+        optionalOptions.add(SINK_PARALLELISM);
+        optionalOptions.add(LookupOptions.CACHE_TYPE);
+        optionalOptions.add(LookupOptions.MAX_RETRIES);
+        optionalOptions.add(LOOKUP_RETRY_INTERVAL);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_MAX_ROWS);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_CACHE_MISSING_KEY);
+        return optionalOptions;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> forwardOptions() {
+        final Set<ConfigOption<?>> forwardOptions = new HashSet<>();
+        forwardOptions.add(SCAN_FETCH_SIZE);
+        forwardOptions.add(SCAN_CURSOR_BATCH_SIZE);
+        forwardOptions.add(SCAN_CURSOR_NO_TIMEOUT);
+        forwardOptions.add(BUFFER_FLUSH_MAX_ROWS);
+        forwardOptions.add(BUFFER_FLUSH_INTERVAL);
+        forwardOptions.add(SINK_MAX_RETRIES);
+        forwardOptions.add(SINK_RETRY_INTERVAL);
+        return forwardOptions;
+    }
+
+    @Override
+    public DynamicTableSource createDynamicTableSource(Context context) {
+        final FactoryUtil.TableFactoryHelper helper =
+                FactoryUtil.createTableFactoryHelper(this, context);
+
+        ReadableConfig options = helper.getOptions();
+        MongoConfiguration config = new MongoConfiguration(options);
+        helper.validate();
+
+        return new MongoDynamicTableSource(
+                getConnectionOptions(config),
+                getReadOptions(config),
+                getLookupCache(options),
+                config.getLookupMaxRetries(),
+                config.getLookupRetryIntervalMs(),
+                context.getPhysicalRowDataType());
+    }
+
+    @Override
+    public DynamicTableSink createDynamicTableSink(Context context) {
+        final FactoryUtil.TableFactoryHelper helper =
+                FactoryUtil.createTableFactoryHelper(this, context);
+
+        MongoConfiguration config = new MongoConfiguration(helper.getOptions());
+        helper.validate();
+
+        ResolvedSchema schema = context.getCatalogTable().getResolvedSchema();
+        SerializableFunction<RowData, BsonValue> keyExtractor =
+                MongoKeyExtractor.createKeyExtractor(schema);
+
+        return new MongoDynamicTableSink(
+                getConnectionOptions(config),
+                getWriteOptions(config),
+                config.getSinkParallelism(),
+                context.getPhysicalRowDataType(),
+                keyExtractor);
+    }
+
+    @Nullable
+    private LookupCache getLookupCache(ReadableConfig tableOptions) {
+        LookupCache cache = null;
+        if (tableOptions
+                .get(LookupOptions.CACHE_TYPE)
+                .equals(LookupOptions.LookupCacheType.PARTIAL)) {
+            cache = DefaultLookupCache.fromConfig(tableOptions);
+        }
+        return cache;
+    }
+
+    private MongoConnectionOptions getConnectionOptions(MongoConfiguration configuration) {
+        return MongoConnectionOptions.builder()
+                .setUri(configuration.getUri())
+                .setDatabase(configuration.getDatabase())
+                .setCollection(configuration.getCollection())
+                .build();
+    }
+
+    private MongoReadOptions getReadOptions(MongoConfiguration configuration) {
+        return MongoReadOptions.builder()
+                .setFetchSize(configuration.getFetchSize())
+                .setCursorBatchSize(configuration.getCursorBatchSize())
+                .setNoCursorTimeout(configuration.isNoCursorTimeout())
+                .setPartitionStrategy(configuration.getPartitionStrategy())
+                .setPartitionSize(configuration.getPartitionSize())
+                .setSamplesPerPartition(configuration.getSamplesPerPartition())
+                .build();
+    }
+
+    private MongoWriteOptions getWriteOptions(MongoConfiguration configuration) {
+        return MongoWriteOptions.builder()
+                .setBatchSize(configuration.getBufferFlushMaxRows())
+                .setBatchIntervalMs(configuration.getBufferFlushIntervalMs())
+                .setMaxRetries(configuration.getSinkMaxRetries())
+                .setRetryIntervalMs(configuration.getSinkRetryIntervalMs())
+                .setDeliveryGuarantee(configuration.getDeliveryGuarantee())
+                .build();
+    }

Review Comment:
   these can be static



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoConnectorOptions.java:
##########
@@ -0,0 +1,143 @@
+/*
+ * 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.annotation.PublicEvolving;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+
+import java.time.Duration;
+
+/**
+ * Base options for the MongoDB connector. Needs to be public so that the {@link
+ * org.apache.flink.table.api.TableDescriptor} can access it.
+ */
+@PublicEvolving
+public class MongoConnectorOptions {
+
+    private MongoConnectorOptions() {}
+
+    public static final ConfigOption<String> URI =
+            ConfigOptions.key("uri")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the connection uri of MongoDB.");
+
+    public static final ConfigOption<String> DATABASE =
+            ConfigOptions.key("database")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the database to read or write of MongoDB.");
+
+    public static final ConfigOption<String> COLLECTION =
+            ConfigOptions.key("collection")
+                    .stringType()
+                    .noDefaultValue()
+                    .withDescription("Specifies the collection to read or write of MongoDB.");
+
+    public static final ConfigOption<Integer> SCAN_FETCH_SIZE =
+            ConfigOptions.key("scan.fetch-size")
+                    .intType()
+                    .defaultValue(2048)
+                    .withDescription(
+                            "Gives the reader a hint as to the number of documents that should be fetched from the database per round-trip when reading. ");
+
+    public static final ConfigOption<Integer> SCAN_CURSOR_BATCH_SIZE =
+            ConfigOptions.key("scan.cursor.batch-size")
+                    .intType()
+                    .defaultValue(0)
+                    .withDescription(
+                            "Specifies the number of documents to return in each batch of the response from the MongoDB instance. Set to 0 to use server's default.");
+
+    public static final ConfigOption<Boolean> SCAN_CURSOR_NO_TIMEOUT =
+            ConfigOptions.key("scan.cursor.no-timeout")
+                    .booleanType()
+                    .defaultValue(true)
+                    .withDescription(
+                            "The server normally times out idle cursors after an inactivity"
+                                    + " period (10 minutes) to prevent excess memory use. Set this option to true to prevent that."
+                                    + " However, if the application takes longer than 30 minutes to process the current batch of documents,"
+                                    + " the session is marked as expired and closed.");
+
+    public static final ConfigOption<PartitionStrategy> SCAN_PARTITION_STRATEGY =
+            ConfigOptions.key("scan.partition.strategy")
+                    .enumType(PartitionStrategy.class)
+                    .defaultValue(PartitionStrategy.DEFAULT)
+                    .withDescription(
+                            "Specifies the partition strategy. Available strategies are single, sample, split-vector, sharded and default.");
+
+    public static final ConfigOption<MemorySize> SCAN_PARTITION_SIZE =
+            ConfigOptions.key("scan.partition.size")
+                    .memoryType()
+                    .defaultValue(MemorySize.parse("64mb"))
+                    .withDescription("Specifies the partition memory size.");
+
+    public static final ConfigOption<Integer> SCAN_PARTITION_SAMPLES =
+            ConfigOptions.key("scan.partition.samples")
+                    .intType()
+                    .defaultValue(10)
+                    .withDescription(
+                            "Specifies the the samples count per partition. It only takes effect when the partition strategy is sample. "
+                                    + "The sample partitioner samples the collection, projects and sorts by the partition fields. "
+                                    + "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<Duration> LOOKUP_RETRY_INTERVAL =
+            ConfigOptions.key("lookup.retry.interval")
+                    .durationType()
+                    .defaultValue(Duration.ofMillis(1000L))
+                    .withDescription(
+                            "Specifies the retry time interval if lookup records from database failed.");
+
+    public static final ConfigOption<Integer> BUFFER_FLUSH_MAX_ROWS =
+            ConfigOptions.key("sink.buffer-flush.max-rows")
+                    .intType()
+                    .defaultValue(1000)
+                    .withDescription(
+                            "Specifies the maximum number of buffered rows per bulk request.");

Review Comment:
   ```suggestion
                               "Specifies the maximum number of buffered rows per batch request.");
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/MongoSinkITCase.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+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.test.junit5.MiniClusterExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 javax.annotation.Nullable;
+
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** IT cases for {@link MongoSink}. */
+@Testcontainers
+public class MongoSinkITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSinkITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    private static final String TEST_DATABASE = "test_sink";
+
+    private static boolean failed;
+
+    private static MongoClient mongoClient;
+
+    @BeforeAll
+    static void setUp() {
+        failed = false;
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void tearDown() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @ParameterizedTest
+    @EnumSource(DeliveryGuarantee.class)
+    void testWriteToMongoWithDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee)
+            throws Exception {
+        final String index = "test-sink-with-delivery-" + deliveryGuarantee;
+        boolean failure = false;
+        try {
+            runTest(index, false, deliveryGuarantee, null);
+        } catch (IllegalArgumentException e) {
+            failure = true;
+            assertThat(deliveryGuarantee).isSameAs(DeliveryGuarantee.EXACTLY_ONCE);
+        } finally {
+            assertThat(failure).isEqualTo(deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE);
+        }
+    }
+
+    @Test
+    void testRecovery() throws Exception {
+        final String index = "test-recovery-mongo-sink";
+        runTest(index, true, new FailingMapper());
+        assertThat(failed).isTrue();
+    }
+
+    private void runTest(
+            String collection,
+            boolean allowRestarts,
+            @Nullable MapFunction<Long, Long> additionalMapper)
+            throws Exception {
+        runTest(collection, allowRestarts, DeliveryGuarantee.AT_LEAST_ONCE, additionalMapper);
+    }
+
+    private void runTest(
+            String collection,
+            boolean allowRestarts,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable MapFunction<Long, Long> additionalMapper)
+            throws Exception {
+
+        final MongoSink<Document> sink =
+                MongoSink.<Document>builder()
+                        .setUri(MONGO_CONTAINER.getConnectionString())
+                        .setDatabase(TEST_DATABASE)
+                        .setCollection(collection)
+                        .setBatchSize(5)
+                        .setDeliveryGuarantee(deliveryGuarantee)
+                        .setSerializationSchema(new AppendOnlySerializationSchema())
+                        .build();
+
+        final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.enableCheckpointing(100L);
+        if (!allowRestarts) {
+            env.setRestartStrategy(RestartStrategies.noRestart());
+        }
+        DataStream<Long> stream = env.fromSequence(1, 5);
+
+        if (additionalMapper != null) {
+            stream = stream.map(additionalMapper);
+        }
+
+        stream.map(new TestMapFunction()).sinkTo(sink);
+        env.execute();
+        assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5);
+    }
+
+    private MongoCollection<Document> collectionOf(String collection) {

Review Comment:
   could be static



##########
flink-sql-connector-mongodb/pom.xml:
##########
@@ -59,11 +59,12 @@ under the License.
 						<configuration>
 							<artifactSet>
 								<includes>
-									<include>org.apache.flink:flink-connector-mongodb</include>
-									<include>org.mongodb:bson</include>
-									<include>org.mongodb:mongodb-driver-sync</include>
-									<include>org.mongodb:mongodb-driver-core</include>
+									<include>*:*</include>
 								</includes>
+								<excludes>
+									<!-- This dependency is not required and will cause problems when shaded. -->
+									<exclude>org.mongodb:bson-record-codec</exclude>

Review Comment:
   Exclude it in flink-connector-mongodb instead. If we don't need to bundle it here, then users will likely not need it as well :)



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoScanSourceSplitReader.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.api.connector.source.SourceReaderContext;
+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.MongoReadOptions;
+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 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;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.project;
+
+/** An split reader implements {@link SplitReader} for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSourceSplitReader implements MongoSourceSplitReader<MongoSourceSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSourceSplitReader.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    private final SourceReaderContext readerContext;
+    @Nullable private final List<String> projectedFields;
+    private final int limit;
+
+    private boolean closed = false;
+    private boolean finished = false;
+    private MongoClient mongoClient;
+    private MongoCursor<BsonDocument> currentCursor;
+    private MongoScanSourceSplit currentSplit;
+
+    public MongoScanSourceSplitReader(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            @Nullable List<String> projectedFields,
+            int limit,
+            SourceReaderContext context) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.projectedFields = projectedFields;
+        this.limit = limit;
+        this.readerContext = context;
+    }
+
+    @Override
+    public RecordsWithSplitIds<BsonDocument> fetch() throws IOException {
+        if (closed) {
+            throw new IllegalStateException("Cannot fetch records from a closed split reader");
+        }
+
+        RecordsBySplits.Builder<BsonDocument> builder = new RecordsBySplits.Builder<>();
+
+        // Return when no split registered to this reader.
+        if (currentSplit == null) {
+            return builder.build();
+        }
+
+        currentCursor = getOrCreateCursor();
+        int fetchSize = readOptions.getFetchSize();
+
+        try {
+            for (int recordNum = 0; recordNum < fetchSize; recordNum++) {
+                if (currentCursor.hasNext()) {
+                    builder.add(currentSplit, currentCursor.next());
+                } else {
+                    builder.addFinishedSplit(currentSplit.splitId());
+                    finished = true;
+                    break;
+                }
+            }
+            return builder.build();
+        } catch (MongoException e) {
+            throw new IOException("Scan records form MongoDB failed", e);
+        } finally {
+            if (finished) {
+                currentSplit = null;
+                releaseCursor();
+            }
+        }
+    }
+
+    @Override
+    public void handleSplitsChanges(SplitsChange<MongoSourceSplit> 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()));
+        }
+
+        MongoSourceSplit sourceSplit = splitsChanges.splits().get(0);
+        if (!(sourceSplit instanceof MongoScanSourceSplit)) {
+            throw new UnsupportedOperationException(
+                    String.format(
+                            "The SourceSplit type of %s is not supported.",
+                            sourceSplit.getClass()));
+        }
+
+        this.currentSplit = (MongoScanSourceSplit) sourceSplit;
+        this.finished = false;
+    }
+
+    @Override
+    public void wakeUp() {}

Review Comment:
   We should adhere to the contract and interrupt the fetch loop if this is called.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoShardedSplitter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.mongodb.MongoException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isShardedCollectionDropped;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readChunks;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readCollectionMetadata;
+
+/**
+ * Sharded Partitioner
+ *
+ * <p>Uses the chunks collection and partitions the collection based on the sharded collections
+ * chunk ranges.
+ *
+ * <p>The following config collections' read privilege is required.
+ *
+ * <ul>
+ *   <li>config.collections
+ *   <li>config.chunks
+ * </ul>
+ */
+@Internal
+public class MongoShardedSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+    public static final MongoShardedSplitter INSTANCE = new MongoShardedSplitter();
+
+    private MongoShardedSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoClient mongoClient = splitContext.getMongoClient();
+
+        List<BsonDocument> chunks;
+        Optional<BsonDocument> collectionMetadata;
+        try {
+            collectionMetadata = readCollectionMetadata(mongoClient, namespace);
+            if (!collectionMetadata.isPresent()) {
+                LOG.error(
+                        "Do sharded split failed, collection {} does not appear to be sharded.",
+                        namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, %s is not a sharded collection.",
+                                namespace));
+            }
+
+            if (isShardedCollectionDropped(collectionMetadata.get())) {
+                LOG.error("Do sharded split failed, collection {} was dropped.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format("Do sharded split failed, %s was dropped.", namespace));
+            }
+
+            chunks = readChunks(mongoClient, collectionMetadata.get());
+            if (chunks.isEmpty()) {
+                LOG.error("Do sharded split failed, chunks of {} is empty.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, chunks of %s is empty.", namespace));
+            }
+        } catch (MongoException e) {
+            LOG.error(
+                    "Read chunks from {} failed with error message: {}", namespace, e.getMessage());
+            throw new FlinkRuntimeException(e);
+        }
+
+        List<MongoScanSourceSplit> sourceSplits = new ArrayList<>(chunks.size());
+        for (int i = 0; i < chunks.size(); i++) {
+            BsonDocument chunk = chunks.get(i);
+            sourceSplits.add(
+                    new MongoScanSourceSplit(
+                            String.format("%s_%d", namespace, i),

Review Comment:
   should we maybe encode the min/max field in here instead of an (arbitrary?) index?



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/MongoSinkITCase.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+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.test.junit5.MiniClusterExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 javax.annotation.Nullable;
+
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** IT cases for {@link MongoSink}. */
+@Testcontainers
+public class MongoSinkITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSinkITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    private static final String TEST_DATABASE = "test_sink";
+
+    private static boolean failed;

Review Comment:
   This isn't a good approach. It's not safe since it's accessed by different threads, it's unclear what it signifies, and TBH I have no idea what the one test that uses it is even asserting with it.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoShardedSplitter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.mongodb.MongoException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isShardedCollectionDropped;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readChunks;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readCollectionMetadata;
+
+/**
+ * Sharded Partitioner
+ *
+ * <p>Uses the chunks collection and partitions the collection based on the sharded collections
+ * chunk ranges.
+ *
+ * <p>The following config collections' read privilege is required.
+ *
+ * <ul>
+ *   <li>config.collections
+ *   <li>config.chunks
+ * </ul>
+ */
+@Internal
+public class MongoShardedSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+    public static final MongoShardedSplitter INSTANCE = new MongoShardedSplitter();
+
+    private MongoShardedSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoClient mongoClient = splitContext.getMongoClient();
+
+        List<BsonDocument> chunks;
+        Optional<BsonDocument> collectionMetadata;
+        try {
+            collectionMetadata = readCollectionMetadata(mongoClient, namespace);
+            if (!collectionMetadata.isPresent()) {
+                LOG.error(
+                        "Do sharded split failed, collection {} does not appear to be sharded.",
+                        namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, %s is not a sharded collection.",
+                                namespace));
+            }
+
+            if (isShardedCollectionDropped(collectionMetadata.get())) {
+                LOG.error("Do sharded split failed, collection {} was dropped.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format("Do sharded split failed, %s was dropped.", namespace));
+            }
+
+            chunks = readChunks(mongoClient, collectionMetadata.get());
+            if (chunks.isEmpty()) {
+                LOG.error("Do sharded split failed, chunks of {} is empty.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, chunks of %s is empty.", namespace));
+            }
+        } catch (MongoException e) {
+            LOG.error(
+                    "Read chunks from {} failed with error message: {}", namespace, e.getMessage());
+            throw new FlinkRuntimeException(e);
+        }
+
+        List<MongoScanSourceSplit> sourceSplits = new ArrayList<>(chunks.size());
+        for (int i = 0; i < chunks.size(); i++) {
+            BsonDocument chunk = chunks.get(i);

Review Comment:
   Does this contain any other fields that would allow users to correlate a Flink split with a MongoDB chunk?



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractorTest.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.table.api.DataTypes;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+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.data.TimestampData;
+
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.Test;
+
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link MongoKeyExtractor}. */
+public class MongoKeyExtractorTest {
+
+    @Test
+    public void testSimpleKey() {
+        ResolvedSchema schema =
+                new ResolvedSchema(
+                        Arrays.asList(
+                                Column.physical("a", DataTypes.BIGINT().notNull()),
+                                Column.physical("b", DataTypes.STRING())),
+                        Collections.emptyList(),
+                        UniqueConstraint.primaryKey("pk", Collections.singletonList("a")));
+
+        Function<RowData, BsonValue> keyExtractor = MongoKeyExtractor.createKeyExtractor(schema);
+
+        BsonValue key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD")));
+        assertThat(key).isEqualTo(new BsonInt64(12L));
+    }
+
+    @Test
+    public void testPrimaryKeyWithReservedId() {
+        ResolvedSchema schema0 =
+                new ResolvedSchema(
+                        Arrays.asList(
+                                Column.physical("_id", DataTypes.STRING().notNull()),
+                                Column.physical("b", DataTypes.STRING())),
+                        Collections.emptyList(),
+                        UniqueConstraint.primaryKey("pk", Collections.singletonList("_id, a")));
+
+        assertThatThrownBy(() -> MongoKeyExtractor.createKeyExtractor(schema0))
+                .isInstanceOf(IllegalArgumentException.class)
+                .hasMessage(
+                        "The primary key should be declared as (_id) when mongo reserved _id field is present");
+
+        ResolvedSchema schema1 =
+                new ResolvedSchema(
+                        Arrays.asList(
+                                Column.physical("_id", DataTypes.STRING().notNull()),
+                                Column.physical("b", DataTypes.STRING())),
+                        Collections.emptyList(),
+                        null);
+
+        assertThatThrownBy(() -> MongoKeyExtractor.createKeyExtractor(schema1))
+                .isInstanceOf(IllegalArgumentException.class)
+                .hasMessage(
+                        "The primary key should be declared as (_id) when mongo reserved _id field is present");
+
+        ResolvedSchema schema2 =
+                new ResolvedSchema(
+                        Arrays.asList(
+                                Column.physical("_id", DataTypes.STRING().notNull()),
+                                Column.physical("b", DataTypes.STRING())),
+                        Collections.emptyList(),
+                        UniqueConstraint.primaryKey("pk", Collections.singletonList("_id")));
+
+        Function<RowData, BsonValue> keyExtractor = MongoKeyExtractor.createKeyExtractor(schema2);
+
+        ObjectId objectId = new ObjectId();
+        BsonValue key =
+                keyExtractor.apply(
+                        GenericRowData.of(
+                                StringData.fromString(objectId.toHexString()),
+                                StringData.fromString("ABCD")));
+        assertThat(key).isEqualTo(new BsonObjectId(objectId));

Review Comment:
   This should be a separate test.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/MongoSinkITCase.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+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.test.junit5.MiniClusterExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 javax.annotation.Nullable;
+
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** IT cases for {@link MongoSink}. */
+@Testcontainers
+public class MongoSinkITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSinkITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .setConfiguration(new Configuration())

Review Comment:
   ```suggestion
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractorTest.java:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.table.api.DataTypes;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+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.data.TimestampData;
+
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.Test;
+
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.function.Function;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link MongoKeyExtractor}. */
+public class MongoKeyExtractorTest {
+
+    @Test
+    public void testSimpleKey() {
+        ResolvedSchema schema =
+                new ResolvedSchema(
+                        Arrays.asList(
+                                Column.physical("a", DataTypes.BIGINT().notNull()),
+                                Column.physical("b", DataTypes.STRING())),
+                        Collections.emptyList(),
+                        UniqueConstraint.primaryKey("pk", Collections.singletonList("a")));
+
+        Function<RowData, BsonValue> keyExtractor = MongoKeyExtractor.createKeyExtractor(schema);
+
+        BsonValue key = keyExtractor.apply(GenericRowData.of(12L, StringData.fromString("ABCD")));
+        assertThat(key).isEqualTo(new BsonInt64(12L));
+    }
+
+    @Test
+    public void testPrimaryKeyWithReservedId() {
+        ResolvedSchema schema0 =
+                new ResolvedSchema(
+                        Arrays.asList(
+                                Column.physical("_id", DataTypes.STRING().notNull()),
+                                Column.physical("b", DataTypes.STRING())),
+                        Collections.emptyList(),
+                        UniqueConstraint.primaryKey("pk", Collections.singletonList("_id, a")));
+
+        assertThatThrownBy(() -> MongoKeyExtractor.createKeyExtractor(schema0))
+                .isInstanceOf(IllegalArgumentException.class)
+                .hasMessage(
+                        "The primary key should be declared as (_id) when mongo reserved _id field is present");

Review Comment:
   This will fail CI since the message was changed. Consider asserting only part of the message (e.g., "ambiguous key") to make this less likely in the future. Don't want tests to fail just because we slightly modified the message (like a typo fix).



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactory.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.table.config.MongoConfiguration;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.factories.DynamicTableSinkFactory;
+import org.apache.flink.table.factories.DynamicTableSourceFactory;
+import org.apache.flink.table.factories.FactoryUtil;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.HashSet;
+import java.util.Set;
+
+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.COLLECTION;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DATABASE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.DELIVERY_GUARANTEE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_BATCH_SIZE;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_CURSOR_NO_TIMEOUT;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SCAN_FETCH_SIZE;
+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.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;
+
+/**
+ * Factory for creating configured instances of {@link MongoDynamicTableSource} and {@link
+ * MongoDynamicTableSink}.
+ */
+@Internal
+public class MongoDynamicTableFactory
+        implements DynamicTableSourceFactory, DynamicTableSinkFactory {
+
+    public static final String IDENTIFIER = "mongodb";
+
+    @Override
+    public String factoryIdentifier() {
+        return IDENTIFIER;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> requiredOptions() {
+        final Set<ConfigOption<?>> requiredOptions = new HashSet<>();
+        requiredOptions.add(URI);
+        requiredOptions.add(DATABASE);
+        requiredOptions.add(COLLECTION);
+        return requiredOptions;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> optionalOptions() {
+        final Set<ConfigOption<?>> optionalOptions = new HashSet<>();
+        optionalOptions.add(SCAN_FETCH_SIZE);
+        optionalOptions.add(SCAN_CURSOR_BATCH_SIZE);
+        optionalOptions.add(SCAN_CURSOR_NO_TIMEOUT);
+        optionalOptions.add(SCAN_PARTITION_STRATEGY);
+        optionalOptions.add(SCAN_PARTITION_SIZE);
+        optionalOptions.add(SCAN_PARTITION_SAMPLES);
+        optionalOptions.add(BUFFER_FLUSH_MAX_ROWS);
+        optionalOptions.add(BUFFER_FLUSH_INTERVAL);
+        optionalOptions.add(DELIVERY_GUARANTEE);
+        optionalOptions.add(SINK_MAX_RETRIES);
+        optionalOptions.add(SINK_RETRY_INTERVAL);
+        optionalOptions.add(SINK_PARALLELISM);
+        optionalOptions.add(LookupOptions.CACHE_TYPE);
+        optionalOptions.add(LookupOptions.MAX_RETRIES);
+        optionalOptions.add(LOOKUP_RETRY_INTERVAL);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_ACCESS);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_EXPIRE_AFTER_WRITE);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_MAX_ROWS);
+        optionalOptions.add(LookupOptions.PARTIAL_CACHE_CACHE_MISSING_KEY);
+        return optionalOptions;
+    }
+
+    @Override
+    public Set<ConfigOption<?>> forwardOptions() {

Review Comment:
   should URI/DATABASE/COLLECTION be included here? Options like these are included by the JdbcDynamicTableFactory.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);
+    }
+
+    @Test
+    public void testShardedSplitter() {
+        MongoNamespace namespace = new MongoNamespace("test_db.test_coll");
+        BsonDocument mockCollectionMetadata = mockCollectionMetadata();
+        List<BsonDocument> mockChunksData = mockChunksData();
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(
+                        MongoReadOptions.builder().build(),
+                        mongoClient,
+                        namespace,
+                        mockCollStats());
+
+        List<MongoScanSourceSplit> expected = new ArrayList<>();
+        for (int i = 0; i < mockChunksData.size(); i++) {
+            BsonDocument mockChunkData = mockChunksData.get(i);
+            expected.add(
+                    new MongoScanSourceSplit(
+                            String.format("%s_%d", namespace, i),
+                            namespace.getDatabaseName(),
+                            namespace.getCollectionName(),
+                            mockChunkData.getDocument(MIN_FIELD),
+                            mockChunkData.getDocument(MAX_FIELD),
+                            mockCollectionMetadata.getDocument(KEY_FIELD)));
+        }
+
+        try (MockedStatic<MongoUtils> util = mockStatic(MongoUtils.class)) {
+            util.when(() -> MongoUtils.readCollectionMetadata(any(), any()))
+                    .thenReturn(Optional.of(mockCollectionMetadata));
+
+            util.when(() -> MongoUtils.readChunks(any(), any())).thenReturn(mockChunksData);
+
+            util.when(() -> MongoUtils.isShardedCollectionDropped(any())).thenReturn(false);
+
+            Collection<MongoScanSourceSplit> actual =
+                    MongoShardedSplitter.INSTANCE.split(splitContext);
+            assertThat(actual).isEqualTo(expected);
+        }
+    }
+
+    private BsonDocument mockCollectionMetadata() {
+        return new BsonDocument()
+                .append(ID_FIELD, new BsonObjectId())
+                .append(UUID_FIELD, new BsonBinary(UUID.randomUUID()))
+                .append(DROPPED_FIELD, BsonBoolean.FALSE)
+                .append(KEY_FIELD, ID_HINT);
+    }
+
+    private List<BsonDocument> mockChunksData() {

Review Comment:
   ```suggestion
       private static List<BsonDocument> mockChunksData() {
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.table.converter.BsonToRowDataConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.LookupFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+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;
+import org.bson.conversions.Bson;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+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.project;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A lookup function for {@link MongoDynamicTableSource}. */
+@Internal
+public class MongoRowDataLookupFunction extends LookupFunction {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoRowDataLookupFunction.class);
+    private static final long serialVersionUID = 1L;
+
+    private final MongoConnectionOptions connectionOptions;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+
+    private final List<String> fieldNames;
+    private final List<String> keyNames;
+
+    private final BsonToRowDataConverters.BsonToRowDataConverter mongoRowConverter;
+    private final RowDataToBsonConverters.RowDataToBsonConverter lookupKeyRowConverter;
+
+    private transient MongoClient mongoClient;
+
+    public MongoRowDataLookupFunction(
+            MongoConnectionOptions connectionOptions,
+            int maxRetries,
+            long retryIntervalMs,
+            List<String> fieldNames,
+            List<DataType> fieldTypes,
+            List<String> keyNames,
+            RowType rowType) {
+        checkNotNull(fieldNames, "No fieldNames supplied.");
+        checkNotNull(fieldTypes, "No fieldTypes supplied.");
+        checkNotNull(keyNames, "No keyNames supplied.");
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.fieldNames = fieldNames;
+        this.mongoRowConverter = BsonToRowDataConverters.createNullableConverter(rowType);
+
+        this.keyNames = keyNames;
+        LogicalType[] keyTypes =
+                this.keyNames.stream()
+                        .map(
+                                s -> {
+                                    checkArgument(
+                                            fieldNames.contains(s),

Review Comment:
   Is this genuinsly something the connector needs to check? I would've assumed that the Table API guarantees this to be the case.



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.MongoDBContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+class MongoE2ECase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    private static final String MONGO_4_0 = "mongo:4.0.10";
+
+    private static final Network NETWORK = Network.newNetwork();
+
+    private final Path sqlConnectorMongoDBJar = ResourceTestUtils.getResource(".*mongodb.jar");
+
+    @Container
+    static final MongoDBContainer MONGO_CONTAINER =
+            new MongoDBContainer(MONGO_4_0)
+                    .withLogConsumer(new Slf4jLogConsumer(LOG))
+                    .withNetwork(NETWORK)
+                    .withNetworkAliases(MONGODB_HOSTNAME);
+
+    private static final TestcontainersSettings TESTCONTAINERS_SETTINGS =
+            TestcontainersSettings.builder()
+                    .logger(LOG)
+                    .network(NETWORK)
+                    .dependsOn(MONGO_CONTAINER)
+                    .build();
+
+    @RegisterExtension
+    public static final FlinkContainers FLINK =
+            FlinkContainers.builder()
+                    .withFlinkContainersSettings(
+                            FlinkContainersSettings.builder().numTaskManagers(2).build())
+                    .withTestcontainersSettings(TESTCONTAINERS_SETTINGS)
+                    .build();
+
+    private static MongoClient mongoClient;
+
+    @BeforeAll
+    static void setUp() throws Exception {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void teardown() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @Test
+    public void testTableApiSourceAndSink() throws Exception {
+        MongoDatabase db = mongoClient.getDatabase("test");
+
+        int ordersCount = 5;
+        List<Document> orders = mockOrders(ordersCount);
+        db.getCollection("orders").insertMany(orders);
+
+        executeSqlStatements(readSqlFile("mongo_e2e.sql"));
+
+        List<Document> ordersBackup = readAllBackupOrders(db, ordersCount);
+
+        assertThat(ordersBackup).containsExactlyInAnyOrderElementsOf(orders);
+    }
+
+    private List<Document> readAllBackupOrders(MongoDatabase db, int expect) throws Exception {
+        Deadline deadline = Deadline.fromNow(Duration.ofSeconds(20));
+        List<Document> backupOrders;
+        do {
+            Thread.sleep(1000);
+            backupOrders = db.getCollection("orders_bak").find().into(new ArrayList<>());
+
+            db.getCollection("orders_bak").find().into(new ArrayList<>());
+        } while (deadline.hasTimeLeft() && backupOrders.size() < expect);
+
+        return backupOrders;
+    }
+
+    private List<Document> mockOrders(int ordersCount) {
+        List<Document> orders = new ArrayList<>();
+        for (int i = 1; i <= ordersCount; i++) {
+            orders.add(
+                    new Document("_id", new ObjectId())
+                            .append("code", "ORDER_" + i)
+                            .append("quantity", ordersCount * 10L));
+        }
+        return orders;
+    }
+
+    private List<String> readSqlFile(final String resourceName) throws Exception {
+        return Files.readAllLines(Paths.get(getClass().getResource("/" + resourceName).toURI()));
+    }
+
+    private void executeSqlStatements(final List<String> sqlLines) throws Exception {

Review Comment:
   can be static, also applies to other utility methods in here.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoSourceITCase.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoJsonDeserializationSchema;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataDeserializationSchema;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+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.MiniClusterExtension;
+import org.apache.flink.util.CollectionUtil;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.params.provider.EnumSource.Mode.EXCLUDE;
+
+/** IT cases for using Mongo Sink. */
+@Testcontainers
+public class MongoSourceITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSourceITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    private static MongoClient mongoClient;
+
+    private static final String TEST_DATABASE = "test_source";
+
+    public static final String TEST_COLLECTION = "mongo_source";
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+
+        MongoCollection<BsonDocument> coll =
+                mongoClient
+                        .getDatabase(TEST_DATABASE)
+                        .getCollection(TEST_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        List<BsonDocument> testRecords = new ArrayList<>();
+        for (int i = 1; i <= 30000; i++) {
+            testRecords.add(createTestData(i));
+            if (testRecords.size() >= 10000) {
+                coll.insertMany(testRecords);
+                testRecords.clear();
+            }
+        }
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @ParameterizedTest
+    @EnumSource(
+            value = PartitionStrategy.class,
+            names = {"SHARDED"},
+            mode = EXCLUDE)
+    public void testPartitionStrategy(PartitionStrategy partitionStrategy) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(2);
+
+        MongoSource<RowData> mongoSource =
+                defaultSourceBuilder()
+                        .setPartitionSize(MemorySize.parse("1mb"))
+                        .setSamplesPerPartition(3)
+                        .setPartitionStrategy(partitionStrategy)
+                        .build();
+
+        List<RowData> results =
+                CollectionUtil.iteratorToList(
+                        env.fromSource(
+                                        mongoSource,
+                                        WatermarkStrategy.noWatermarks(),
+                                        "MongoDB-Source")
+                                .executeAndCollect());
+
+        assertThat(results).hasSize(30000);
+    }
+
+    @Test
+    public void testLimit() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(2);
+
+        MongoSource<RowData> mongoSource = defaultSourceBuilder().setLimit(100).build();
+
+        List<RowData> results =
+                CollectionUtil.iteratorToList(
+                        env.fromSource(
+                                        mongoSource,
+                                        WatermarkStrategy.noWatermarks(),
+                                        "MongoDB-Source")
+                                .executeAndCollect());
+
+        assertThat(results).hasSize(100);
+    }
+
+    @Test
+    public void testProject() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(2);

Review Comment:
   This shouldn't be necessary. The mini cluster automatically defaults to the maximum parallelism that is can support with the given number of tms and slots_per_tm.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+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.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test for {@link MongoDynamicTableSource} and {@link MongoDynamicTableSink} created by {@link
+ * MongoDynamicTableFactory}.
+ */
+public class MongoDynamicTableFactoryTest {
+
+    private static final ResolvedSchema SCHEMA =
+            new ResolvedSchema(
+                    Arrays.asList(
+                            Column.physical("aaa", DataTypes.INT().notNull()),
+                            Column.physical("bbb", DataTypes.STRING().notNull()),
+                            Column.physical("ccc", DataTypes.DOUBLE()),
+                            Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
+                            Column.physical("eee", DataTypes.TIMESTAMP(3))),
+                    Collections.emptyList(),
+                    UniqueConstraint.primaryKey("name", Arrays.asList("bbb", "aaa")));
+
+    @Test
+    public void testMongoCommonProperties() {
+        Map<String, String> properties = getRequiredOptions();
+
+        // validation for source
+        DynamicTableSource actualSource = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expectedSource =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+        assertThat(actualSource).isEqualTo(expectedSource);
+
+        // validation for sink
+        DynamicTableSink actualSink = createTableSink(SCHEMA, properties);
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+        MongoDynamicTableSink expectedSink =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+        assertThat(actualSink).isEqualTo(expectedSink);
+    }
+
+    @Test
+    public void testMongoReadProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "1024");
+        properties.put("scan.cursor.batch-size", "2048");
+        properties.put("scan.cursor.no-timeout", "false");
+        properties.put("scan.partition.strategy", "split-vector");
+        properties.put("scan.partition.size", "128m");
+        properties.put("scan.partition.samples", "5");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoReadOptions readOptions =
+                MongoReadOptions.builder()
+                        .setFetchSize(1024)
+                        .setCursorBatchSize(2048)
+                        .setNoCursorTimeout(false)
+                        .setPartitionStrategy(PartitionStrategy.SPLIT_VECTOR)
+                        .setPartitionSize(MemorySize.ofMebiBytes(128))
+                        .setSamplesPerPartition(5)
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        readOptions,
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoLookupProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("lookup.cache", "PARTIAL");
+        properties.put("lookup.partial-cache.expire-after-write", "10s");
+        properties.put("lookup.partial-cache.expire-after-access", "20s");
+        properties.put("lookup.partial-cache.cache-missing-key", "false");
+        properties.put("lookup.partial-cache.max-rows", "15213");
+        properties.put("lookup.max-retries", "10");
+        properties.put("lookup.retry.interval", "20ms");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        DefaultLookupCache.fromConfig(Configuration.fromMap(properties)),
+                        10,
+                        20,
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.buffer-flush.max-rows", "1001");
+        properties.put("sink.buffer-flush.interval", "2min");
+        properties.put("sink.delivery-guarantee", "at-least-once");
+        properties.put("sink.max-retries", "5");
+        properties.put("sink.retry.interval", "2s");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoWriteOptions writeOptions =
+                MongoWriteOptions.builder()
+                        .setBatchSize(1001)
+                        .setBatchIntervalMs(TimeUnit.MINUTES.toMillis(2))
+                        .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+                        .setMaxRetries(5)
+                        .setRetryIntervalMs(TimeUnit.SECONDS.toMillis(2))
+                        .build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkWithParallelism() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.parallelism", "2");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        2,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoValidation() {
+        // fetch size lower than 1
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "0");
+
+        Map<String, String> finalProperties1 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties1))
+                .hasStackTraceContaining("The fetch size must be larger than 0.");
+
+        // cursor batch size lower than 0
+        properties = getRequiredOptions();
+        properties.put("scan.cursor.batch-size", "-1");
+
+        Map<String, String> finalProperties2 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties2))
+                .hasStackTraceContaining("The cursor batch size must be larger than or equal to 0");
+
+        // partition memory size lower than 1mb
+        properties = getRequiredOptions();
+        properties.put("scan.partition.size", "900kb");
+
+        Map<String, String> finalProperties3 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties3))
+                .hasStackTraceContaining(
+                        "The partition size must be larger than or equals to 1mb.");

Review Comment:
   ```suggestion
                           "The partition size must be larger than or equal to 1mb.");
   ```
   Also applies to other error messages.



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.MongoDBContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+class MongoE2ECase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    private static final String MONGO_4_0 = "mongo:4.0.10";

Review Comment:
   This should re-use the MongoTestUtils.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/MongoSinkITCase.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+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.test.junit5.MiniClusterExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 javax.annotation.Nullable;
+
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** IT cases for {@link MongoSink}. */
+@Testcontainers
+public class MongoSinkITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSinkITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    private static final String TEST_DATABASE = "test_sink";
+
+    private static boolean failed;
+
+    private static MongoClient mongoClient;
+
+    @BeforeAll
+    static void setUp() {
+        failed = false;
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void tearDown() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @ParameterizedTest
+    @EnumSource(DeliveryGuarantee.class)
+    void testWriteToMongoWithDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee)
+            throws Exception {
+        final String index = "test-sink-with-delivery-" + deliveryGuarantee;
+        boolean failure = false;
+        try {
+            runTest(index, false, deliveryGuarantee, null);
+        } catch (IllegalArgumentException e) {
+            failure = true;
+            assertThat(deliveryGuarantee).isSameAs(DeliveryGuarantee.EXACTLY_ONCE);

Review Comment:
   The exactly-once case adds way too much complexity to this test for something that isn't even supported.



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.MongoDBContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+class MongoE2ECase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    private static final String MONGO_4_0 = "mongo:4.0.10";
+
+    private static final Network NETWORK = Network.newNetwork();
+
+    private final Path sqlConnectorMongoDBJar = ResourceTestUtils.getResource(".*mongodb.jar");
+
+    @Container
+    static final MongoDBContainer MONGO_CONTAINER =
+            new MongoDBContainer(MONGO_4_0)
+                    .withLogConsumer(new Slf4jLogConsumer(LOG))
+                    .withNetwork(NETWORK)
+                    .withNetworkAliases(MONGODB_HOSTNAME);
+
+    private static final TestcontainersSettings TESTCONTAINERS_SETTINGS =
+            TestcontainersSettings.builder()
+                    .logger(LOG)
+                    .network(NETWORK)
+                    .dependsOn(MONGO_CONTAINER)
+                    .build();
+
+    @RegisterExtension
+    public static final FlinkContainers FLINK =
+            FlinkContainers.builder()
+                    .withFlinkContainersSettings(
+                            FlinkContainersSettings.builder().numTaskManagers(2).build())
+                    .withTestcontainersSettings(TESTCONTAINERS_SETTINGS)
+                    .build();
+
+    private static MongoClient mongoClient;
+
+    @BeforeAll
+    static void setUp() throws Exception {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void teardown() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @Test
+    public void testTableApiSourceAndSink() throws Exception {
+        MongoDatabase db = mongoClient.getDatabase("test");
+
+        int ordersCount = 5;
+        List<Document> orders = mockOrders(ordersCount);
+        db.getCollection("orders").insertMany(orders);
+
+        executeSqlStatements(readSqlFile("mongo_e2e.sql"));
+
+        List<Document> ordersBackup = readAllBackupOrders(db, ordersCount);
+
+        assertThat(ordersBackup).containsExactlyInAnyOrderElementsOf(orders);
+    }
+
+    private List<Document> readAllBackupOrders(MongoDatabase db, int expect) throws Exception {
+        Deadline deadline = Deadline.fromNow(Duration.ofSeconds(20));
+        List<Document> backupOrders;
+        do {
+            Thread.sleep(1000);
+            backupOrders = db.getCollection("orders_bak").find().into(new ArrayList<>());
+
+            db.getCollection("orders_bak").find().into(new ArrayList<>());

Review Comment:
   ```suggestion
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/MongoSinkITCase.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+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.test.junit5.MiniClusterExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 javax.annotation.Nullable;
+
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** IT cases for {@link MongoSink}. */
+@Testcontainers
+public class MongoSinkITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSinkITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    private static final String TEST_DATABASE = "test_sink";
+
+    private static boolean failed;
+
+    private static MongoClient mongoClient;
+
+    @BeforeAll
+    static void setUp() {
+        failed = false;
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void tearDown() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @ParameterizedTest
+    @EnumSource(DeliveryGuarantee.class)
+    void testWriteToMongoWithDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee)
+            throws Exception {
+        final String index = "test-sink-with-delivery-" + deliveryGuarantee;
+        boolean failure = false;
+        try {
+            runTest(index, false, deliveryGuarantee, null);
+        } catch (IllegalArgumentException e) {
+            failure = true;
+            assertThat(deliveryGuarantee).isSameAs(DeliveryGuarantee.EXACTLY_ONCE);
+        } finally {
+            assertThat(failure).isEqualTo(deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE);
+        }
+    }
+
+    @Test
+    void testRecovery() throws Exception {
+        final String index = "test-recovery-mongo-sink";
+        runTest(index, true, new FailingMapper());
+        assertThat(failed).isTrue();
+    }
+
+    private void runTest(
+            String collection,
+            boolean allowRestarts,
+            @Nullable MapFunction<Long, Long> additionalMapper)
+            throws Exception {
+        runTest(collection, allowRestarts, DeliveryGuarantee.AT_LEAST_ONCE, additionalMapper);
+    }
+
+    private void runTest(

Review Comment:
   This kind of testing where you have a central test method with various hooks for other tests typically just makes things harder to maintain in the long run.
   
   Instead, move shared code into utility methods (e.g., createSink(), assertResults()) and have the test cases contain more logic, even if some of it is duplicated.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoRowDataLookupFunction.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.table.converter.BsonToRowDataConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.functions.FunctionContext;
+import org.apache.flink.table.functions.LookupFunction;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+
+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;
+import org.bson.conversions.Bson;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+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.project;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A lookup function for {@link MongoDynamicTableSource}. */
+@Internal
+public class MongoRowDataLookupFunction extends LookupFunction {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoRowDataLookupFunction.class);
+    private static final long serialVersionUID = 1L;
+
+    private final MongoConnectionOptions connectionOptions;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+
+    private final List<String> fieldNames;
+    private final List<String> keyNames;
+
+    private final BsonToRowDataConverters.BsonToRowDataConverter mongoRowConverter;
+    private final RowDataToBsonConverters.RowDataToBsonConverter lookupKeyRowConverter;
+
+    private transient MongoClient mongoClient;
+
+    public MongoRowDataLookupFunction(
+            MongoConnectionOptions connectionOptions,
+            int maxRetries,
+            long retryIntervalMs,
+            List<String> fieldNames,
+            List<DataType> fieldTypes,
+            List<String> keyNames,
+            RowType rowType) {
+        checkNotNull(fieldNames, "No fieldNames supplied.");
+        checkNotNull(fieldTypes, "No fieldTypes supplied.");
+        checkNotNull(keyNames, "No keyNames supplied.");
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.fieldNames = fieldNames;
+        this.mongoRowConverter = BsonToRowDataConverters.createNullableConverter(rowType);
+
+        this.keyNames = keyNames;
+        LogicalType[] keyTypes =
+                this.keyNames.stream()
+                        .map(
+                                s -> {
+                                    checkArgument(
+                                            fieldNames.contains(s),
+                                            "keyName %s can't find in fieldNames %s.",
+                                            s,
+                                            fieldNames);
+                                    return fieldTypes.get(fieldNames.indexOf(s)).getLogicalType();
+                                })
+                        .toArray(LogicalType[]::new);
+
+        this.lookupKeyRowConverter =
+                RowDataToBsonConverters.createNullableConverter(
+                        RowType.of(keyTypes, keyNames.toArray(new String[0])));
+    }
+
+    @Override
+    public void open(FunctionContext context) {
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    /**
+     * This is a lookup method which is called by Flink framework in runtime.
+     *
+     * @param keyRow lookup keys
+     */
+    @Override
+    public Collection<RowData> lookup(RowData keyRow) {
+        for (int retry = 0; retry <= maxRetries; retry++) {
+            try {
+                BsonDocument lookupValues = (BsonDocument) lookupKeyRowConverter.convert(keyRow);
+
+                List<Bson> filters =
+                        keyNames.stream()
+                                .map(name -> eq(name, lookupValues.get(name)))
+                                .collect(Collectors.toList());
+                Bson query = and(filters);
+
+                Bson projection = project(fieldNames);
+
+                try (MongoCursor<BsonDocument> cursor =
+                        getMongoCollection().find(query).projection(projection).cursor()) {
+                    ArrayList<RowData> rows = new ArrayList<>();
+                    while (cursor.hasNext()) {
+                        RowData row = (RowData) mongoRowConverter.convert(cursor.next());
+                        rows.add(row);
+                    }
+                    rows.trimToSize();
+                    return rows;
+                }
+            } catch (MongoException e) {
+                LOG.debug("MongoDB lookup error, retry times = {}", retry, e);
+                if (retry >= maxRetries) {

Review Comment:
   ```suggestion
                   if (retry == maxRetries) {
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriterITCase.java:
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.DeleteOneModel;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.UpdateOneModel;
+import com.mongodb.client.model.UpdateOptions;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+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.util.Optional;
+import java.util.OptionalLong;
+
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreNotWritten;
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link MongoWriter}. */
+@Testcontainers
+public class MongoWriterITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriterITCase.class);
+
+    private static final String TEST_DATABASE = "test_writer";
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    private static MongoClient mongoClient;
+    private static MetricListener metricListener;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void setUp() {
+        metricListener = new MetricListener();
+    }
+
+    @Test
+    void testWriteOnBulkFlush() throws Exception {
+        final String collection = "test-bulk-flush-without-checkpoint";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 5;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+
+            // Ignore flush on checkpoint
+            writer.flush(false);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3, 4);
+
+            // Trigger flush
+            writer.write(buildMessage(5), null);
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5);
+
+            writer.write(buildMessage(6), null);
+            assertThatIdsAreNotWritten(collectionOf(collection), 6);
+
+            // Force flush
+            writer.doBulkWrite();
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5, 6);
+        }
+    }
+
+    @Test
+    void testWriteOnBatchIntervalFlush() throws Exception {
+        final String collection = "test-bulk-flush-with-interval";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = -1;
+        final int batchIntervalMs = 1000;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+            writer.doBulkWrite();
+        }
+
+        assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4);
+    }
+
+    @Test
+    void testWriteOnCheckpoint() throws Exception {
+        final String collection = "test-bulk-flush-with-checkpoint";
+        final boolean flushOnCheckpoint = true;
+        final int batchSize = -1;
+        final int batchIntervalMs = -1;
+
+        // Enable flush on checkpoint
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3);
+
+            // Trigger flush
+            writer.flush(false);
+
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3);
+        }
+    }
+
+    @Test
+    void testIncrementRecordsSendMetric() throws Exception {
+        final String collection = "test-inc-records-send";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Counter> recordsSend =
+                    metricListener.getCounter(MetricNames.NUM_RECORDS_SEND);
+            writer.write(buildMessage(1), null);
+            // Update existing index
+            writer.write(buildMessage(2, "u"), null);
+            // Delete index
+            writer.write(buildMessage(3, "d"), null);
+
+            writer.doBulkWrite();
+
+            assertThat(recordsSend.isPresent()).isTrue();
+            assertThat(recordsSend.get().getCount()).isEqualTo(3L);
+        }
+    }
+
+    @Test
+    void testCurrentSendTime() throws Exception {
+        final String collection = "test-current-send-time";
+        boolean flushOnCheckpoint = false;

Review Comment:
   ```suggestion
           final boolean flushOnCheckpoint = false;
   ```
   for consistency



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSink.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.MongoSink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataSerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A {@link DynamicTableSink} for MongoDB. */
+@Internal
+public class MongoDynamicTableSink implements DynamicTableSink {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    @Nullable private final Integer parallelism;
+    private final DataType physicalRowDataType;
+    private final SerializableFunction<RowData, BsonValue> keyExtractor;
+
+    public MongoDynamicTableSink(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            @Nullable Integer parallelism,
+            DataType physicalRowDataType,
+            SerializableFunction<RowData, BsonValue> keyExtractor) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.parallelism = parallelism;
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.keyExtractor = checkNotNull(keyExtractor);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        // UPSERT mode
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {

Review Comment:
   I'd rather explicitly list the supported RowKinds, instead of implicitly supporting anything that isn't `UPDATE_BEFORE`. If a new RowKind is added we want to explicitly opt-in to that.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactoryTest.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+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.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.cache.DefaultLookupCache;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/**
+ * Test for {@link MongoDynamicTableSource} and {@link MongoDynamicTableSink} created by {@link
+ * MongoDynamicTableFactory}.
+ */
+public class MongoDynamicTableFactoryTest {
+
+    private static final ResolvedSchema SCHEMA =
+            new ResolvedSchema(
+                    Arrays.asList(
+                            Column.physical("aaa", DataTypes.INT().notNull()),
+                            Column.physical("bbb", DataTypes.STRING().notNull()),
+                            Column.physical("ccc", DataTypes.DOUBLE()),
+                            Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
+                            Column.physical("eee", DataTypes.TIMESTAMP(3))),
+                    Collections.emptyList(),
+                    UniqueConstraint.primaryKey("name", Arrays.asList("bbb", "aaa")));
+
+    @Test
+    public void testMongoCommonProperties() {
+        Map<String, String> properties = getRequiredOptions();
+
+        // validation for source
+        DynamicTableSource actualSource = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expectedSource =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+        assertThat(actualSource).isEqualTo(expectedSource);
+
+        // validation for sink
+        DynamicTableSink actualSink = createTableSink(SCHEMA, properties);
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+        MongoDynamicTableSink expectedSink =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+        assertThat(actualSink).isEqualTo(expectedSink);
+    }
+
+    @Test
+    public void testMongoReadProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "1024");
+        properties.put("scan.cursor.batch-size", "2048");
+        properties.put("scan.cursor.no-timeout", "false");
+        properties.put("scan.partition.strategy", "split-vector");
+        properties.put("scan.partition.size", "128m");
+        properties.put("scan.partition.samples", "5");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoReadOptions readOptions =
+                MongoReadOptions.builder()
+                        .setFetchSize(1024)
+                        .setCursorBatchSize(2048)
+                        .setNoCursorTimeout(false)
+                        .setPartitionStrategy(PartitionStrategy.SPLIT_VECTOR)
+                        .setPartitionSize(MemorySize.ofMebiBytes(128))
+                        .setSamplesPerPartition(5)
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        readOptions,
+                        null,
+                        LookupOptions.MAX_RETRIES.defaultValue(),
+                        LOOKUP_RETRY_INTERVAL.defaultValue().toMillis(),
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoLookupProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("lookup.cache", "PARTIAL");
+        properties.put("lookup.partial-cache.expire-after-write", "10s");
+        properties.put("lookup.partial-cache.expire-after-access", "20s");
+        properties.put("lookup.partial-cache.cache-missing-key", "false");
+        properties.put("lookup.partial-cache.max-rows", "15213");
+        properties.put("lookup.max-retries", "10");
+        properties.put("lookup.retry.interval", "20ms");
+
+        DynamicTableSource actual = createTableSource(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoDynamicTableSource expected =
+                new MongoDynamicTableSource(
+                        connectionOptions,
+                        MongoReadOptions.builder().build(),
+                        DefaultLookupCache.fromConfig(Configuration.fromMap(properties)),
+                        10,
+                        20,
+                        SCHEMA.toPhysicalRowDataType());
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkProperties() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.buffer-flush.max-rows", "1001");
+        properties.put("sink.buffer-flush.interval", "2min");
+        properties.put("sink.delivery-guarantee", "at-least-once");
+        properties.put("sink.max-retries", "5");
+        properties.put("sink.retry.interval", "2s");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+        MongoWriteOptions writeOptions =
+                MongoWriteOptions.builder()
+                        .setBatchSize(1001)
+                        .setBatchIntervalMs(TimeUnit.MINUTES.toMillis(2))
+                        .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
+                        .setMaxRetries(5)
+                        .setRetryIntervalMs(TimeUnit.SECONDS.toMillis(2))
+                        .build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        null,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoSinkWithParallelism() {
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("sink.parallelism", "2");
+
+        DynamicTableSink actual = createTableSink(SCHEMA, properties);
+
+        MongoConnectionOptions connectionOptions =
+                MongoConnectionOptions.builder()
+                        .setUri("mongodb://127.0.0.1:27017")
+                        .setDatabase("test_db")
+                        .setCollection("test_coll")
+                        .build();
+
+        MongoWriteOptions writeOptions = MongoWriteOptions.builder().build();
+
+        MongoDynamicTableSink expected =
+                new MongoDynamicTableSink(
+                        connectionOptions,
+                        writeOptions,
+                        2,
+                        SCHEMA.toPhysicalRowDataType(),
+                        MongoKeyExtractor.createKeyExtractor(SCHEMA));
+
+        assertThat(actual).isEqualTo(expected);
+    }
+
+    @Test
+    public void testMongoValidation() {
+        // fetch size lower than 1
+        Map<String, String> properties = getRequiredOptions();
+        properties.put("scan.fetch-size", "0");
+
+        Map<String, String> finalProperties1 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties1))
+                .hasStackTraceContaining("The fetch size must be larger than 0.");
+
+        // cursor batch size lower than 0
+        properties = getRequiredOptions();
+        properties.put("scan.cursor.batch-size", "-1");
+
+        Map<String, String> finalProperties2 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties2))
+                .hasStackTraceContaining("The cursor batch size must be larger than or equal to 0");
+
+        // partition memory size lower than 1mb
+        properties = getRequiredOptions();
+        properties.put("scan.partition.size", "900kb");
+
+        Map<String, String> finalProperties3 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties3))
+                .hasStackTraceContaining(
+                        "The partition size must be larger than or equals to 1mb.");
+
+        // samples per partition lower than 1
+        properties = getRequiredOptions();
+        properties.put("scan.partition.samples", "0");
+
+        Map<String, String> finalProperties4 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties4))
+                .hasStackTraceContaining("The samples per partition must be larger than 0.");
+
+        // lookup retry times shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("lookup.max-retries", "-1");
+        Map<String, String> finalProperties5 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties5))
+                .hasStackTraceContaining(
+                        "The 'lookup.max-retries' must be larger than or equals to 0.");
+
+        // lookup retry interval shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("lookup.retry.interval", "0ms");
+        Map<String, String> finalProperties6 = properties;
+        assertThatThrownBy(() -> createTableSource(SCHEMA, finalProperties6))
+                .hasStackTraceContaining("The 'lookup.retry.interval' must be larger than 0.");
+
+        // sink retries shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("sink.max-retries", "-1");
+        Map<String, String> finalProperties7 = properties;
+        assertThatThrownBy(() -> createTableSink(SCHEMA, finalProperties7))
+                .hasStackTraceContaining(
+                        "The sink max retry times must be larger than or equal to 0.");
+
+        // sink retry interval shouldn't be negative
+        properties = getRequiredOptions();
+        properties.put("sink.retry.interval", "0ms");
+        Map<String, String> finalProperties8 = properties;
+        assertThatThrownBy(() -> createTableSink(SCHEMA, finalProperties8))
+                .hasStackTraceContaining(
+                        "The retry interval (in milliseconds) must be larger than 0.");
+
+        // sink buffered rows shouldn't be smaller than 1

Review Comment:
   ```suggestion
           // sink buffered rows should be larger than 0
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoSourceITCase.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoJsonDeserializationSchema;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataDeserializationSchema;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+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.MiniClusterExtension;
+import org.apache.flink.util.CollectionUtil;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.params.provider.EnumSource.Mode.EXCLUDE;
+
+/** IT cases for using Mongo Sink. */
+@Testcontainers
+public class MongoSourceITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSourceITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    private static MongoClient mongoClient;
+
+    private static final String TEST_DATABASE = "test_source";
+
+    public static final String TEST_COLLECTION = "mongo_source";
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+
+        MongoCollection<BsonDocument> coll =
+                mongoClient
+                        .getDatabase(TEST_DATABASE)
+                        .getCollection(TEST_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        List<BsonDocument> testRecords = new ArrayList<>();
+        for (int i = 1; i <= 30000; i++) {
+            testRecords.add(createTestData(i));
+            if (testRecords.size() >= 10000) {
+                coll.insertMany(testRecords);
+                testRecords.clear();
+            }
+        }
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @ParameterizedTest
+    @EnumSource(
+            value = PartitionStrategy.class,
+            names = {"SHARDED"},
+            mode = EXCLUDE)
+    public void testPartitionStrategy(PartitionStrategy partitionStrategy) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(2);
+
+        MongoSource<RowData> mongoSource =
+                defaultSourceBuilder()
+                        .setPartitionSize(MemorySize.parse("1mb"))
+                        .setSamplesPerPartition(3)
+                        .setPartitionStrategy(partitionStrategy)
+                        .build();
+
+        List<RowData> results =
+                CollectionUtil.iteratorToList(
+                        env.fromSource(
+                                        mongoSource,
+                                        WatermarkStrategy.noWatermarks(),
+                                        "MongoDB-Source")
+                                .executeAndCollect());
+
+        assertThat(results).hasSize(30000);
+    }
+
+    @Test
+    public void testLimit() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(2);
+
+        MongoSource<RowData> mongoSource = defaultSourceBuilder().setLimit(100).build();
+
+        List<RowData> results =
+                CollectionUtil.iteratorToList(
+                        env.fromSource(
+                                        mongoSource,
+                                        WatermarkStrategy.noWatermarks(),
+                                        "MongoDB-Source")
+                                .executeAndCollect());
+
+        assertThat(results).hasSize(100);
+    }
+
+    @Test
+    public void testProject() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(2);
+
+        MongoSource<String> mongoSource =
+                MongoSource.<String>builder()
+                        .setUri(MONGO_CONTAINER.getConnectionString())
+                        .setDatabase(TEST_DATABASE)
+                        .setCollection(TEST_COLLECTION)
+                        .setProjectedFields("f0")
+                        .setDeserializationSchema(new MongoJsonDeserializationSchema())
+                        .build();
+
+        List<String> results =
+                CollectionUtil.iteratorToList(
+                        env.fromSource(
+                                        mongoSource,
+                                        WatermarkStrategy.noWatermarks(),
+                                        "MongoDB-Source")
+                                .executeAndCollect());
+
+        assertThat(results).hasSize(30000);
+        assertThat(Document.parse(results.get(0))).containsOnlyKeys("f0");
+    }
+
+    private MongoSourceBuilder<RowData> defaultSourceBuilder() {
+        ResolvedSchema schema = defaultSourceSchema();
+        RowType rowType = (RowType) schema.toPhysicalRowDataType().getLogicalType();
+        TypeInformation<RowData> typeInfo = InternalTypeInfo.of(rowType);
+
+        return MongoSource.<RowData>builder()
+                .setUri(MONGO_CONTAINER.getConnectionString())
+                .setDatabase(TEST_DATABASE)
+                .setCollection(TEST_COLLECTION)
+                .setDeserializationSchema(new MongoRowDataDeserializationSchema(rowType, typeInfo));
+    }
+
+    private ResolvedSchema defaultSourceSchema() {
+        return ResolvedSchema.of(
+                Column.physical("f0", DataTypes.INT()), Column.physical("f1", DataTypes.STRING()));
+    }
+
+    private static BsonDocument createTestData(int id) {

Review Comment:
   can be static etc



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoSourceITCase.java:
##########
@@ -0,0 +1,208 @@
+/*
+ * 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.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoJsonDeserializationSchema;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataDeserializationSchema;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+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.MiniClusterExtension;
+import org.apache.flink.util.CollectionUtil;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import org.apache.commons.lang3.RandomStringUtils;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.params.provider.EnumSource.Mode.EXCLUDE;
+
+/** IT cases for using Mongo Sink. */
+@Testcontainers
+public class MongoSourceITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSourceITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    private static MongoClient mongoClient;
+
+    private static final String TEST_DATABASE = "test_source";
+
+    public static final String TEST_COLLECTION = "mongo_source";
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+
+        MongoCollection<BsonDocument> coll =
+                mongoClient
+                        .getDatabase(TEST_DATABASE)
+                        .getCollection(TEST_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        List<BsonDocument> testRecords = new ArrayList<>();
+        for (int i = 1; i <= 30000; i++) {
+            testRecords.add(createTestData(i));
+            if (testRecords.size() >= 10000) {
+                coll.insertMany(testRecords);
+                testRecords.clear();
+            }
+        }
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @ParameterizedTest
+    @EnumSource(
+            value = PartitionStrategy.class,
+            names = {"SHARDED"},
+            mode = EXCLUDE)
+    public void testPartitionStrategy(PartitionStrategy partitionStrategy) throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(2);
+
+        MongoSource<RowData> mongoSource =
+                defaultSourceBuilder()
+                        .setPartitionSize(MemorySize.parse("1mb"))
+                        .setSamplesPerPartition(3)
+                        .setPartitionStrategy(partitionStrategy)
+                        .build();
+
+        List<RowData> results =
+                CollectionUtil.iteratorToList(
+                        env.fromSource(
+                                        mongoSource,
+                                        WatermarkStrategy.noWatermarks(),
+                                        "MongoDB-Source")
+                                .executeAndCollect());
+
+        assertThat(results).hasSize(30000);
+    }
+
+    @Test
+    public void testLimit() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(2);
+
+        MongoSource<RowData> mongoSource = defaultSourceBuilder().setLimit(100).build();
+
+        List<RowData> results =
+                CollectionUtil.iteratorToList(
+                        env.fromSource(
+                                        mongoSource,
+                                        WatermarkStrategy.noWatermarks(),
+                                        "MongoDB-Source")
+                                .executeAndCollect());
+
+        assertThat(results).hasSize(100);
+    }
+
+    @Test
+    public void testProject() throws Exception {
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(2);
+
+        MongoSource<String> mongoSource =
+                MongoSource.<String>builder()
+                        .setUri(MONGO_CONTAINER.getConnectionString())
+                        .setDatabase(TEST_DATABASE)
+                        .setCollection(TEST_COLLECTION)
+                        .setProjectedFields("f0")
+                        .setDeserializationSchema(new MongoJsonDeserializationSchema())
+                        .build();
+
+        List<String> results =
+                CollectionUtil.iteratorToList(
+                        env.fromSource(
+                                        mongoSource,
+                                        WatermarkStrategy.noWatermarks(),
+                                        "MongoDB-Source")
+                                .executeAndCollect());
+
+        assertThat(results).hasSize(30000);

Review Comment:
   magic number; move 30000 into a constant



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+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.MongoReadOptions;
+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;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
+import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** A {@link DynamicTableSource} for MongoDB. */
+@Internal
+public class MongoDynamicTableSource
+        implements ScanTableSource,
+                LookupTableSource,
+                SupportsProjectionPushDown,
+                SupportsLimitPushDown {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    @Nullable private final LookupCache lookupCache;
+    private final int lookupMaxRetries;
+    private final long lookupRetryIntervalMs;
+    private DataType physicalRowDataType;
+    private int limit = -1;
+
+    public MongoDynamicTableSource(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            @Nullable LookupCache lookupCache,
+            int lookupMaxRetries,
+            long lookupRetryIntervalMs,
+            DataType physicalRowDataType) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.lookupCache = lookupCache;
+        checkArgument(
+                lookupMaxRetries >= 0,
+                String.format(
+                        "The '%s' must be larger than or equals to 0.",
+                        LookupOptions.MAX_RETRIES.key()));
+        checkArgument(
+                lookupRetryIntervalMs > 0,
+                String.format("The '%s' must be larger than 0.", LOOKUP_RETRY_INTERVAL.key()));
+        this.lookupMaxRetries = lookupMaxRetries;
+        this.lookupRetryIntervalMs = lookupRetryIntervalMs;
+        this.physicalRowDataType = physicalRowDataType;
+    }
+
+    @Override
+    public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) {
+        final List<String> keyNames = new ArrayList<>(context.getKeys().length);
+        for (int i = 0; i < context.getKeys().length; i++) {
+            int[] innerKeyArr = context.getKeys()[i];
+            Preconditions.checkArgument(
+                    innerKeyArr.length == 1, "MongoDB only support non-nested look up keys yet");
+            keyNames.add(DataType.getFieldNames(physicalRowDataType).get(innerKeyArr[0]));
+        }

Review Comment:
   ```suggestion
           for (int[] innerKeyArr : context.getKeys()) {
               Preconditions.checkArgument(
                       innerKeyArr.length == 1, "MongoDB only support non-nested look up keys yet");
               keyNames.add(DataType.getFieldNames(physicalRowDataType).get(innerKeyArr[0]));
           }
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/split/MongoScanSourceSplitReader.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.api.connector.source.SourceReaderContext;
+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.MongoReadOptions;
+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 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;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.project;
+
+/** An split reader implements {@link SplitReader} for {@link MongoScanSourceSplit}. */
+@Internal
+public class MongoScanSourceSplitReader implements MongoSourceSplitReader<MongoSourceSplit> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoScanSourceSplitReader.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    private final SourceReaderContext readerContext;
+    @Nullable private final List<String> projectedFields;
+    private final int limit;
+
+    private boolean closed = false;
+    private boolean finished = false;
+    private MongoClient mongoClient;
+    private MongoCursor<BsonDocument> currentCursor;
+    private MongoScanSourceSplit currentSplit;
+
+    public MongoScanSourceSplitReader(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            @Nullable List<String> projectedFields,
+            int limit,
+            SourceReaderContext context) {
+        this.connectionOptions = connectionOptions;
+        this.readOptions = readOptions;
+        this.projectedFields = projectedFields;
+        this.limit = limit;
+        this.readerContext = context;
+    }
+
+    @Override
+    public RecordsWithSplitIds<BsonDocument> fetch() throws IOException {
+        if (closed) {
+            throw new IllegalStateException("Cannot fetch records from a closed split reader");
+        }
+
+        RecordsBySplits.Builder<BsonDocument> builder = new RecordsBySplits.Builder<>();
+
+        // Return when no split registered to this reader.
+        if (currentSplit == null) {
+            return builder.build();
+        }
+
+        currentCursor = getOrCreateCursor();
+        int fetchSize = readOptions.getFetchSize();
+
+        try {
+            for (int recordNum = 0; recordNum < fetchSize; recordNum++) {
+                if (currentCursor.hasNext()) {
+                    builder.add(currentSplit, currentCursor.next());
+                } else {
+                    builder.addFinishedSplit(currentSplit.splitId());
+                    finished = true;
+                    break;
+                }
+            }

Review Comment:
   Is this loop actually that much more efficient than using a fetchSize of 1?
   If we aren't actively pulling multiple records from mongodb _at once_, then what is the real benefit here?
   Seems like we are potentially blocking the reader thread for a long time (if everything next() call issues another request to mongodb).
   



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoShardedSplitter.java:
##########
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.mongodb.MongoException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isShardedCollectionDropped;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readChunks;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readCollectionMetadata;
+
+/**
+ * Sharded Partitioner
+ *
+ * <p>Uses the chunks collection and partitions the collection based on the sharded collections
+ * chunk ranges.
+ *
+ * <p>The following config collections' read privilege is required.
+ *
+ * <ul>
+ *   <li>config.collections
+ *   <li>config.chunks
+ * </ul>
+ */
+@Internal
+public class MongoShardedSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+    public static final MongoShardedSplitter INSTANCE = new MongoShardedSplitter();
+
+    private MongoShardedSplitter() {}
+
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoClient mongoClient = splitContext.getMongoClient();
+
+        List<BsonDocument> chunks;
+        Optional<BsonDocument> collectionMetadata;
+        try {
+            collectionMetadata = readCollectionMetadata(mongoClient, namespace);
+            if (!collectionMetadata.isPresent()) {
+                LOG.error(
+                        "Do sharded split failed, collection {} does not appear to be sharded.",
+                        namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, %s is not a sharded collection.",
+                                namespace));
+            }
+
+            if (isShardedCollectionDropped(collectionMetadata.get())) {
+                LOG.error("Do sharded split failed, collection {} was dropped.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format("Do sharded split failed, %s was dropped.", namespace));
+            }
+
+            chunks = readChunks(mongoClient, collectionMetadata.get());
+            if (chunks.isEmpty()) {
+                LOG.error("Do sharded split failed, chunks of {} is empty.", namespace);
+                throw new FlinkRuntimeException(
+                        String.format(
+                                "Do sharded split failed, chunks of %s is empty.", namespace));
+            }

Review Comment:
   Is this actually en error case? What if the collection is just empty?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSink.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.MongoSink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataSerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A {@link DynamicTableSink} for MongoDB. */
+@Internal
+public class MongoDynamicTableSink implements DynamicTableSink {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    @Nullable private final Integer parallelism;
+    private final DataType physicalRowDataType;
+    private final SerializableFunction<RowData, BsonValue> keyExtractor;
+
+    public MongoDynamicTableSink(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            @Nullable Integer parallelism,
+            DataType physicalRowDataType,
+            SerializableFunction<RowData, BsonValue> keyExtractor) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.parallelism = parallelism;
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.keyExtractor = checkNotNull(keyExtractor);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        // UPSERT mode
+        ChangelogMode.Builder builder = ChangelogMode.newBuilder();
+        for (RowKind kind : requestedMode.getContainedKinds()) {
+            if (kind != RowKind.UPDATE_BEFORE) {
+                builder.addContainedKind(kind);
+            }
+        }
+        return builder.build();
+    }
+
+    @Override
+    public SinkRuntimeProvider getSinkRuntimeProvider(Context context) {
+        final RowDataToBsonConverter rowDataToBsonConverter =
+                RowDataToBsonConverters.createNullableConverter(
+                        physicalRowDataType.getLogicalType());
+
+        final MongoRowDataSerializationSchema serializationSchema =
+                new MongoRowDataSerializationSchema(rowDataToBsonConverter, keyExtractor);
+
+        MongoSink<RowData> mongoSink =

Review Comment:
   ```suggestion
           final MongoSink<RowData> mongoSink =
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java:
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.utils.MongoValidationUtils;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.catalog.UniqueConstraint;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.data.utils.ProjectedRowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonObjectId;
+import org.bson.BsonValue;
+import org.bson.types.ObjectId;
+
+import java.util.Optional;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** An extractor for a MongoDB key from a {@link RowData}. */
+@Internal
+public class MongoKeyExtractor implements SerializableFunction<RowData, BsonValue> {
+
+    public static final String RESERVED_ID = ID_FIELD;
+
+    private static final AppendOnlyKeyExtractor APPEND_ONLY_KEY_EXTRACTOR =
+            new AppendOnlyKeyExtractor();
+
+    private final LogicalType primaryKeyType;
+
+    private final int[] primaryKeyIndexes;
+
+    private final RowDataToBsonConverters.RowDataToBsonConverter primaryKeyConverter;
+
+    private MongoKeyExtractor(LogicalType primaryKeyType, int[] primaryKeyIndexes) {
+        this.primaryKeyType = primaryKeyType;
+        this.primaryKeyIndexes = primaryKeyIndexes;
+        this.primaryKeyConverter = RowDataToBsonConverters.createNullableConverter(primaryKeyType);
+    }
+
+    @Override
+    public BsonValue apply(RowData rowData) {
+        BsonValue keyValue;
+        if (isCompoundPrimaryKey(primaryKeyIndexes)) {
+            RowData keyRow = ProjectedRowData.from(primaryKeyIndexes).replaceRow(rowData);
+            keyValue = primaryKeyConverter.convert(keyRow);
+        } else {
+            RowData.FieldGetter fieldGetter =
+                    RowData.createFieldGetter(primaryKeyType, primaryKeyIndexes[0]);
+            keyValue = primaryKeyConverter.convert(fieldGetter.getFieldOrNull(rowData));
+            if (keyValue.isString()) {
+                String keyString = keyValue.asString().getValue();
+                // Try to restore MongoDB's ObjectId from string.
+                if (ObjectId.isValid(keyString)) {
+                    keyValue = new BsonObjectId(new ObjectId(keyString));
+                }
+            }
+        }
+        return checkNotNull(keyValue, "Primary key value is null of RowData: " + rowData);
+    }
+
+    public static SerializableFunction<RowData, BsonValue> createKeyExtractor(
+            ResolvedSchema resolvedSchema) {
+
+        Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey();
+        int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+        Optional<Column> reservedId = resolvedSchema.getColumn(RESERVED_ID);
+
+        // Primary key is not declared and reserved _id is not present.

Review Comment:
   should be fine then



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/MongoSinkITCase.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * 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.sink;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+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.test.junit5.MiniClusterExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 javax.annotation.Nullable;
+
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** IT cases for {@link MongoSink}. */
+@Testcontainers
+public class MongoSinkITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSinkITCase.class);
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    private static final String TEST_DATABASE = "test_sink";
+
+    private static boolean failed;

Review Comment:
   Rather use a BlockerSync that you passed via a `SharedObjects` to the FailingMapper constructor.
   
   Have the test thread wait until notifyCheckpointComplete was called, then unblock the mapper.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/splitter/MongoShardedSplitterTest.java:
##########
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.splitter;
+
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoShardedSplitter;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.MongoSplitContext;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.BsonObjectId;
+import org.bson.BsonString;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.MockedStatic;
+import org.mockito.MockitoAnnotations;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_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.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mockStatic;
+
+/** Unit tests for {@link MongoShardedSplitter}. */
+@ExtendWith(TestLoggerExtension.class)
+public class MongoShardedSplitterTest {
+
+    @Mock private MongoClient mongoClient;
+
+    @BeforeEach
+    public void setUp() {
+        MockitoAnnotations.initMocks(this);

Review Comment:
   > Also, I'm a bit confused about these functional wrappers, using them we might still need to mock their results. Is there something wrong with my understanding?
   
   With _mocking_ I specifically refer to the usage of mockito, not things like `mockCollectionMetadata`.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriterITCase.java:
##########
@@ -0,0 +1,426 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.operators.ProcessingTimeService;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.mongodb.MongoTestUtil;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.Gauge;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.metrics.groups.OperatorIOMetricGroup;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.metrics.testutils.MetricListener;
+import org.apache.flink.runtime.mailbox.SyncMailboxExecutor;
+import org.apache.flink.runtime.metrics.MetricNames;
+import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup;
+import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.util.UserCodeClassLoader;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.DeleteOneModel;
+import com.mongodb.client.model.InsertOneModel;
+import com.mongodb.client.model.UpdateOneModel;
+import com.mongodb.client.model.UpdateOptions;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.bson.Document;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+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.util.Optional;
+import java.util.OptionalLong;
+
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreNotWritten;
+import static org.apache.flink.connector.mongodb.MongoTestUtil.assertThatIdsAreWritten;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link MongoWriter}. */
+@Testcontainers
+public class MongoWriterITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriterITCase.class);
+
+    private static final String TEST_DATABASE = "test_writer";
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setConfiguration(new Configuration())
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    private static MongoClient mongoClient;
+    private static MetricListener metricListener;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void setUp() {
+        metricListener = new MetricListener();
+    }
+
+    @Test
+    void testWriteOnBulkFlush() throws Exception {
+        final String collection = "test-bulk-flush-without-checkpoint";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 5;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+
+            // Ignore flush on checkpoint
+            writer.flush(false);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3, 4);
+
+            // Trigger flush
+            writer.write(buildMessage(5), null);
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5);
+
+            writer.write(buildMessage(6), null);
+            assertThatIdsAreNotWritten(collectionOf(collection), 6);
+
+            // Force flush
+            writer.doBulkWrite();
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4, 5, 6);
+        }
+    }
+
+    @Test
+    void testWriteOnBatchIntervalFlush() throws Exception {
+        final String collection = "test-bulk-flush-with-interval";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = -1;
+        final int batchIntervalMs = 1000;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+            writer.write(buildMessage(4), null);
+            writer.doBulkWrite();
+        }
+
+        assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3, 4);
+    }
+
+    @Test
+    void testWriteOnCheckpoint() throws Exception {
+        final String collection = "test-bulk-flush-with-checkpoint";
+        final boolean flushOnCheckpoint = true;
+        final int batchSize = -1;
+        final int batchIntervalMs = -1;
+
+        // Enable flush on checkpoint
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+            writer.write(buildMessage(3), null);
+
+            assertThatIdsAreNotWritten(collectionOf(collection), 1, 2, 3);
+
+            // Trigger flush
+            writer.flush(false);
+
+            assertThatIdsAreWritten(collectionOf(collection), 1, 2, 3);
+        }
+    }
+
+    @Test
+    void testIncrementRecordsSendMetric() throws Exception {
+        final String collection = "test-inc-records-send";
+        final boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Counter> recordsSend =
+                    metricListener.getCounter(MetricNames.NUM_RECORDS_SEND);
+            writer.write(buildMessage(1), null);
+            // Update existing index
+            writer.write(buildMessage(2, "u"), null);
+            // Delete index
+            writer.write(buildMessage(3, "d"), null);
+
+            writer.doBulkWrite();
+
+            assertThat(recordsSend.isPresent()).isTrue();
+            assertThat(recordsSend.get().getCount()).isEqualTo(3L);
+        }
+    }
+
+    @Test
+    void testCurrentSendTime() throws Exception {
+        final String collection = "test-current-send-time";
+        boolean flushOnCheckpoint = false;
+        final int batchSize = 2;
+        final int batchIntervalMs = -1;
+
+        try (final MongoWriter<Document> writer =
+                createWriter(collection, batchSize, batchIntervalMs, flushOnCheckpoint)) {
+            final Optional<Gauge<Long>> currentSendTime =
+                    metricListener.getGauge("currentSendTime");
+
+            writer.write(buildMessage(1), null);
+            writer.write(buildMessage(2), null);
+
+            writer.doBulkWrite();
+
+            assertThat(currentSendTime.isPresent()).isTrue();
+            assertThat(currentSendTime.get().getValue()).isGreaterThan(0L);
+        }
+    }
+
+    @Test
+    void testSinkContext() throws Exception {
+        final String collection = "test-sink-context";
+        boolean flushOnCheckpoint = false;

Review Comment:
   ```suggestion
           final boolean flushOnCheckpoint = false;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1030535503


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+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.SINK_MAX_RETRIES;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int batchSize;
+    private final long batchIntervalMs;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    private MongoWriteOptions(
+            int batchSize,
+            long batchIntervalMs,
+            int maxRetries,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee) {
+        this.batchSize = batchSize;
+        this.batchIntervalMs = batchIntervalMs;
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+    }
+
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    public long getBatchIntervalMs() {
+        return batchIntervalMs;
+    }
+
+    public int getMaxRetries() {
+        return maxRetries;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return batchSize == that.batchSize
+                && batchIntervalMs == that.batchIntervalMs
+                && maxRetries == that.maxRetries
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                batchSize, batchIntervalMs, maxRetries, retryIntervalMs, deliveryGuarantee);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int batchSize = BUFFER_FLUSH_MAX_ROWS.defaultValue();
+        private long batchIntervalMs = BUFFER_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetries = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param batchSize the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBatchSize(int batchSize) {
+            checkArgument(
+                    batchSize == -1 || batchSize > 0,
+                    "Max number of batch size must be larger than 0.");
+            this.batchSize = batchSize;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param batchIntervalMs the batch flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBatchIntervalMs(long batchIntervalMs) {
+            checkArgument(
+                    batchIntervalMs == -1 || batchIntervalMs >= 0,
+                    "The batch flush interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");

Review Comment:
   We should put this somewhere into the docs.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025554030


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+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.MongoReadOptions;
+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;
+import org.apache.flink.table.connector.Projection;
+import org.apache.flink.table.connector.source.DynamicTableSource;
+import org.apache.flink.table.connector.source.LookupTableSource;
+import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.SourceProvider;
+import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
+import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
+import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
+import org.apache.flink.table.connector.source.lookup.LookupOptions;
+import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
+import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.LOOKUP_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** A {@link DynamicTableSource} for MongoDB. */
+@Internal
+public class MongoDynamicTableSource
+        implements ScanTableSource,
+                LookupTableSource,
+                SupportsProjectionPushDown,
+                SupportsLimitPushDown {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoReadOptions readOptions;
+    @Nullable private final LookupCache lookupCache;
+    private final int lookupMaxRetries;
+    private final long lookupRetryIntervalMs;
+    private DataType physicalRowDataType;
+    private int limit = -1;

Review Comment:
   I think we should hold it because we need to pass parameters when constructing `LookupRuntimeProvider` and `ScanRuntimeProvider`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1033362006


##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.MongoDBContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+class MongoE2ECase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    private static final String MONGO_4_0 = "mongo:4.0.10";

Review Comment:
   yes. Previously no one was using the test-jar.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] twalthr commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
twalthr commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1052181728


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactory.java:
##########
@@ -151,16 +146,11 @@ public DynamicTableSink createDynamicTableSink(Context context) {
         MongoConfiguration config = new MongoConfiguration(helper.getOptions());
         helper.validate();
 
-        ResolvedSchema schema = context.getCatalogTable().getResolvedSchema();
-        SerializableFunction<RowData, BsonValue> keyExtractor =
-                MongoKeyExtractor.createKeyExtractor(schema);
-
         return new MongoDynamicTableSink(
                 getConnectionOptions(config),
                 getWriteOptions(config),
                 config.getSinkParallelism(),
-                context.getPhysicalRowDataType(),
-                keyExtractor);
+                context.getCatalogTable().getResolvedSchema());

Review Comment:
   Personally, I would keep out the catalog's schema from the sink. The sink has no schema but a data type. Let's use a boolean flag in the constructor `isUpsert`.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/converter/MongoConvertersTest.java:
##########
@@ -146,28 +146,31 @@ public void testConvertBsonToRowData() {
                 GenericRowData.of(
                         StringData.fromString(oid.toHexString()),
                         StringData.fromString("string"),
-                        StringData.fromString(uuid.toString()),
+                        StringData.fromString(
+                                "{\"_value\": {\"$binary\": {\"base64\": \"gR+qXamERr+L0IyxvO9daQ==\", \"subType\": \"04\"}}}"),
                         2,
                         3L,
                         4.1d,
                         DecimalData.fromBigDecimal(new BigDecimal("5.1"), 10, 2),
                         false,
                         TimestampData.fromEpochMillis(now.getEpochSecond() * 1000),
+                        TimestampData.fromEpochMillis(now.toEpochMilli()),
                         StringData.fromString(
-                                OffsetDateTime.ofInstant(
-                                                Instant.ofEpochMilli(now.toEpochMilli()),
-                                                ZoneOffset.UTC)
-                                        .format(ISO_OFFSET_DATE_TIME)),
-                        StringData.fromString("/^9$/i"),
-                        StringData.fromString("function() { return 10; }"),
-                        StringData.fromString("function() { return 11; }"),
-                        StringData.fromString("12"),
-                        StringData.fromString(oid.toHexString()),
+                                "{\"_value\": {\"$regularExpression\": {\"pattern\": \"^9$\", \"options\": \"i\"}}}"),

Review Comment:
   The question whether we need to customize a JsonWriter depends on the round-trip story. Is it possible to read a RegExp as string and write it out as a string again that MongoDB could correctly classify as RegExp again?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1052209447


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/converter/MongoConvertersTest.java:
##########
@@ -146,28 +146,31 @@ public void testConvertBsonToRowData() {
                 GenericRowData.of(
                         StringData.fromString(oid.toHexString()),
                         StringData.fromString("string"),
-                        StringData.fromString(uuid.toString()),
+                        StringData.fromString(
+                                "{\"_value\": {\"$binary\": {\"base64\": \"gR+qXamERr+L0IyxvO9daQ==\", \"subType\": \"04\"}}}"),
                         2,
                         3L,
                         4.1d,
                         DecimalData.fromBigDecimal(new BigDecimal("5.1"), 10, 2),
                         false,
                         TimestampData.fromEpochMillis(now.getEpochSecond() * 1000),
+                        TimestampData.fromEpochMillis(now.toEpochMilli()),
                         StringData.fromString(
-                                OffsetDateTime.ofInstant(
-                                                Instant.ofEpochMilli(now.toEpochMilli()),
-                                                ZoneOffset.UTC)
-                                        .format(ISO_OFFSET_DATE_TIME)),
-                        StringData.fromString("/^9$/i"),
-                        StringData.fromString("function() { return 10; }"),
-                        StringData.fromString("function() { return 11; }"),
-                        StringData.fromString("12"),
-                        StringData.fromString(oid.toHexString()),
+                                "{\"_value\": {\"$regularExpression\": {\"pattern\": \"^9$\", \"options\": \"i\"}}}"),

Review Comment:
   The current bson api only supports the conversion of bson document. For a single bson value, we need to customize `JsonWriter` and `JsonReader`.
   So in the previous implementation, we used a _value to wrap a single bson value as a bson document, so that we can parse them easily.
   
   ```java
   import org.bson.BsonDocument;
   import org.bson.BsonRegularExpression;
   import org.bson.json.JsonMode;
   import org.bson.json.JsonWriterSettings;
   import org.junit.jupiter.api.Test;
   
   import static org.junit.jupiter.api.Assertions.assertEquals;
   
   public class JsonConversionTest {
   
       @Test
       public void bsonToJsonTest() {
           BsonRegularExpression original = new BsonRegularExpression("regex", "i");
           BsonDocument wrapped = new BsonDocument("_value", original);
   
           String json = wrapped.toJson(JsonWriterSettings.builder().outputMode(JsonMode.EXTENDED).build());
           // {"_value": {"$regularExpression": {"pattern": "regex", "options": "i"}}}
           System.out.println(json);
   
           BsonDocument parsed = BsonDocument.parse(json);
           BsonRegularExpression parsedRegularExpression = parsed.getRegularExpression("_value");
   
           assertEquals(parsedRegularExpression, original);
       }
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1052217885


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/converter/MongoConvertersTest.java:
##########
@@ -146,28 +146,31 @@ public void testConvertBsonToRowData() {
                 GenericRowData.of(
                         StringData.fromString(oid.toHexString()),
                         StringData.fromString("string"),
-                        StringData.fromString(uuid.toString()),
+                        StringData.fromString(
+                                "{\"_value\": {\"$binary\": {\"base64\": \"gR+qXamERr+L0IyxvO9daQ==\", \"subType\": \"04\"}}}"),
                         2,
                         3L,
                         4.1d,
                         DecimalData.fromBigDecimal(new BigDecimal("5.1"), 10, 2),
                         false,
                         TimestampData.fromEpochMillis(now.getEpochSecond() * 1000),
+                        TimestampData.fromEpochMillis(now.toEpochMilli()),
                         StringData.fromString(
-                                OffsetDateTime.ofInstant(
-                                                Instant.ofEpochMilli(now.toEpochMilli()),
-                                                ZoneOffset.UTC)
-                                        .format(ISO_OFFSET_DATE_TIME)),
-                        StringData.fromString("/^9$/i"),
-                        StringData.fromString("function() { return 10; }"),
-                        StringData.fromString("function() { return 11; }"),
-                        StringData.fromString("12"),
-                        StringData.fromString(oid.toHexString()),
+                                "{\"_value\": {\"$regularExpression\": {\"pattern\": \"^9$\", \"options\": \"i\"}}}"),

Review Comment:
   No matter which form it is, it cannot be directly handed over to MongoDB for processing. The bson document stored in mongodb needs to be converted into `BsonDocument`. We need to extract and convert the `RowData` fileds stored as string types, and finally convert the entire `RowData` into `BsonDocument`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1018264629


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoValidationUtils.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.DistinctType;
+import org.apache.flink.table.types.logical.LogicalTypeFamily;
+import org.apache.flink.table.types.logical.LogicalTypeRoot;
+
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/** Utility methods for validating MongoDB properties. */
+@Internal
+public class MongoValidationUtils {
+    private static final Set<LogicalTypeRoot> ALLOWED_PRIMARY_KEY_TYPES = new LinkedHashSet<>();
+
+    static {
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.CHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.VARCHAR);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BOOLEAN);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DECIMAL);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TINYINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.SMALLINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTEGER);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.BIGINT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.FLOAT);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.DOUBLE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_YEAR_MONTH);
+        ALLOWED_PRIMARY_KEY_TYPES.add(LogicalTypeRoot.INTERVAL_DAY_TIME);
+    }
+
+    /**
+     * Checks that the table does not have a primary key defined on illegal types. In MongoDB the
+     * primary key is used to calculate the MongoDB document id, which is a string of up to 1024
+     * bytes. It cannot have whitespaces. As of now it is calculated by concatenating the fields.

Review Comment:
   Hi @zentol, 
   
   Thanks for pointing out the inaccuracies here. The javadocs have been modified to describe the details of extracting primary keys in detail.
   
   ------------
   
   That method checks that the table does not have a primary key defined on illegal types. The illegal types are mostly LogicalTypeFamily.COLLECTION types and LogicalTypeRoot.RAW type and other types that cannot be converted to BsonType by `RowDataToBsonConverters`.
   
   In MongoDB the primary key is used to calculate the MongoDB document _id, which may be of any BsonType other than a BsonType.ARRAY. Its value must be unique and immutable in the collection.
   
   MongoDB creates a unique index on the _id field during the creation of a collection. There are also some constraints on the primary key index. For more detailed introduction, you can refer to [Index Key Limit](https://www.mongodb.com/docs/manual/reference/limits/#mongodb-limit-Index-Key-Limit).
   
   - Before MongoDB 4.2, the total size of an index entry, which can include structural overhead depending on the BSON type, must be less than 1024 bytes.
   - Starting in version 4.2, MongoDB removes the Index Key Limit.
   
   As of now it is extracted by `MongoKeyExtractor` according to the primary key specified by the Flink table schema.
   
   - When there's only a single field in the specified primary key, we convert the field data to bson value as _id of the corresponding document.
   - When there's multiple fields in the specified primary key, we convert and composite these fields into a BSON OBJECT as the _id of the corresponding document.
   
   For example, 
   If have a primary key with single field
   ```sql
   CREATE TABLE T1 (
     f1 BITINT,
     f2 SRING,
     f3 FLOAT,
     PRIMARY KEY (f1) NOT ENFORCED
   ) WITH (
     'connector' = 'mongodb'
   );
   ```
   The extracted _id will be the form like:
   ```javascript
   {
     _id: 1
     f2: "v2",
     f3: 3.0
   }
   ```
   __________
   
   If have a primary key with multiple fields
   ```sql
   CREATE TABLE T2 (
     f1 BITINT,
     f2 SRING,
     f3 FLOAT,
     PRIMARY KEY (f1, f2) NOT ENFORCED
   ) WITH (
     'connector' = 'mongodb'
   );
   ```
   The extracted _id will be the form like:
   ```javascript
   {
     // object type is acceptable for _id in MongoDB
     _id: {
       f1: 1,
       f2: "v2"
     },
     f2: "v2",
     f3: 3.0
   }
   ```
   
   ----------
   
   For why use the object type to combine fields into _id instead of concatenating these fields into a string like ES, the reason is that we don't need to handle all types to string conversions.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1022831114


##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,157 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!--using mongodb shade jar to execute end-to-end test-->

Review Comment:
   ```suggestion
   		<!-- Use fat jar so we don't need to create a user-jar. -->
   ```
   This doesn't explain _why_ we're doing this.



##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,157 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!--using mongodb shade jar to execute end-to-end test-->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-sql-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<profiles>
+		<profile>
+			<id>run-end-to-end-tests</id>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<executions>
+							<execution>
+								<id>end-to-end-tests</id>
+								<phase>integration-test</phase>
+								<goals>
+									<goal>test</goal>
+								</goals>
+								<configuration>
+									<includes>
+										<include>**/*.*</include>
+									</includes>
+									<!-- E2E tests must not access flink-dist concurrently. -->
+									<forkCount>1</forkCount>
+									<systemPropertyVariables>
+										<moduleDir>${project.basedir}</moduleDir>
+									</systemPropertyVariables>
+								</configuration>
+							</execution>
+						</executions>
+					</plugin>
+				</plugins>
+			</build>
+		</profile>
+	</profiles>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-deploy-plugin</artifactId>
+				<configuration>
+					<skip>true</skip>
+				</configuration>
+			</plugin>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-surefire-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>default-test</id>
+						<phase>none</phase>
+					</execution>
+					<execution>
+						<id>integration-tests</id>
+						<phase>none</phase>
+					</execution>
+				</executions>
+			</plugin>

Review Comment:
   How expensive are the e2e tests? I think we could just run them as part of the integration-tests phase.
   (== remove/inline run-end-to-end-tests profile)



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsEqual.equalTo;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+@ExtendWith({TestLoggerExtension.class})
+public class MongoE2ECase extends TestLogger {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    public static final String MONGO_4_0 = "mongo:4.0.10";

Review Comment:
   ```suggestion
       private static final String MONGO_4_0 = "mongo:4.0.10";
   ```



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsEqual.equalTo;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+@ExtendWith({TestLoggerExtension.class})
+public class MongoE2ECase extends TestLogger {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    public static final String MONGO_4_0 = "mongo:4.0.10";
+
+    private static final Network NETWORK = Network.newNetwork();
+
+    private final Path sqlConnectorMongoDBJar = ResourceTestUtils.getResource(".*mongodb.jar");
+
+    @Container
+    public static final MongoDBContainer MONGO_CONTAINER =
+            new MongoDBContainer(MONGO_4_0)
+                    .withLogConsumer(new Slf4jLogConsumer(LOG))
+                    .withNetwork(NETWORK)
+                    .withNetworkAliases(MONGODB_HOSTNAME);
+
+    public static final TestcontainersSettings TESTCONTAINERS_SETTINGS =

Review Comment:
   ```suggestion
       private static final TestcontainersSettings TESTCONTAINERS_SETTINGS =
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable
+    public static BsonDocument readCollectionMetadata(
+            MongoClient mongoClient, MongoNamespace namespace) {
+        MongoCollection<BsonDocument> collection =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(COLLECTIONS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        return collection
+                .find(eq(ID_FIELD, namespace.getFullName()))
+                .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD))
+                .first();
+    }
+
+    public static boolean isValidShardedCollection(BsonDocument collectionMetadata) {
+        return collectionMetadata != null
+                && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue();
+    }
+
+    public static List<BsonDocument> readChunks(
+            MongoClient mongoClient, BsonDocument collectionMetadata) {
+        MongoCollection<BsonDocument> chunks =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(CHUNKS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        Bson filter =
+                or(
+                        new BsonDocument(NAMESPACE_FIELD, collectionMetadata.get(ID_FIELD)),
+                        // MongoDB 4.9.0 removed ns field of config.chunks collection, using

Review Comment:
   This raises an interesting question on compatibility. Which versions do we intend to supported / which are still actively maintained by MongoDB?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+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.SINK_MAX_RETRIES;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int batchSize;
+    private final long batchIntervalMs;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    private MongoWriteOptions(
+            int batchSize,
+            long batchIntervalMs,
+            int maxRetries,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee) {
+        this.batchSize = batchSize;
+        this.batchIntervalMs = batchIntervalMs;
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+    }
+
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    public long getBatchIntervalMs() {
+        return batchIntervalMs;
+    }
+
+    public int getMaxRetries() {
+        return maxRetries;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return batchSize == that.batchSize
+                && batchIntervalMs == that.batchIntervalMs
+                && maxRetries == that.maxRetries
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                batchSize, batchIntervalMs, maxRetries, retryIntervalMs, deliveryGuarantee);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int batchSize = BUFFER_FLUSH_MAX_ROWS.defaultValue();
+        private long batchIntervalMs = BUFFER_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetries = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.

Review Comment:
   ```suggestion
            * Sets the maximum number of actions to buffer for each batch request. You can pass -1 to
            * disable batching.
   ```
   For consistency. Let's try to avoid specific values in the javadocs because they will be out-dated quite quickly. maybe point to the ConnectorOptions instead.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Serializable, Closeable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
+
+    private final MongoReadOptions readOptions;
+    private final boolean limitPushedDown;
+    private final MongoClient mongoClient;
+
+    public MongoSplitters(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            boolean limitPushedDown) {
+        this.readOptions = readOptions;
+        this.limitPushedDown = limitPushedDown;
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    public Collection<MongoScanSourceSplit> split(MongoNamespace namespace) {
+        BsonDocument collStats = MongoUtils.collStats(mongoClient, namespace);
+        if (!isCommandSucceed(collStats)) {
+            LOG.error(
+                    "Execute command collStats failed: {}",
+                    collStats.getString(ERROR_MESSAGE_FIELD));
+            throw new IllegalStateException(String.format("Collection not found %s", namespace));
+        }
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(readOptions, mongoClient, namespace, collStats);
+
+        if (limitPushedDown) {
+            LOG.info("Limit {} is applied, using single splitter", limitPushedDown);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }

Review Comment:
   Do we assume that if a limit was applied the number of records that we consume is so low that splitting doesn't make sense, or is there another reason for why we don't try to split things?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Serializable, Closeable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
+
+    private final MongoReadOptions readOptions;
+    private final boolean limitPushedDown;
+    private final MongoClient mongoClient;
+
+    public MongoSplitters(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            boolean limitPushedDown) {
+        this.readOptions = readOptions;
+        this.limitPushedDown = limitPushedDown;
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    public Collection<MongoScanSourceSplit> split(MongoNamespace namespace) {
+        BsonDocument collStats = MongoUtils.collStats(mongoClient, namespace);
+        if (!isCommandSucceed(collStats)) {
+            LOG.error(
+                    "Execute command collStats failed: {}",
+                    collStats.getString(ERROR_MESSAGE_FIELD));
+            throw new IllegalStateException(String.format("Collection not found %s", namespace));

Review Comment:
   Why couldn't it be that the database is missing?



##########
flink-connector-mongodb-e2e-tests/pom.xml:
##########
@@ -0,0 +1,157 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-connector-mongodb-e2e-tests</artifactId>
+	<name>Flink : E2E Tests : MongoDB</name>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-test-utils</artifactId>
+			<version>${flink.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<!--using mongodb shade jar to execute end-to-end test-->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-sql-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mongodb</groupId>
+			<artifactId>mongodb-driver-sync</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>mongodb</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<profiles>
+		<profile>
+			<id>run-end-to-end-tests</id>
+			<build>
+				<plugins>
+					<plugin>
+						<groupId>org.apache.maven.plugins</groupId>
+						<artifactId>maven-surefire-plugin</artifactId>
+						<executions>
+							<execution>
+								<id>end-to-end-tests</id>
+								<phase>integration-test</phase>
+								<goals>
+									<goal>test</goal>
+								</goals>
+								<configuration>
+									<includes>
+										<include>**/*.*</include>
+									</includes>
+									<!-- E2E tests must not access flink-dist concurrently. -->

Review Comment:
   This _should_ no longer a problem since we work on copies.



##########
flink-sql-connector-mongodb/pom.xml:
##########
@@ -0,0 +1,93 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<artifactId>flink-connector-mongodb-parent</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.0-SNAPSHOT</version>
+	</parent>
+
+	<artifactId>flink-sql-connector-mongodb</artifactId>
+	<name>Flink : Connectors : SQL : MongoDB</name>
+
+	<packaging>jar</packaging>
+
+	<properties>
+		<japicmp.skip>true</japicmp.skip>
+	</properties>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-connector-mongodb</artifactId>
+			<version>${project.version}</version>
+		</dependency>
+	</dependencies>
+
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-shade-plugin</artifactId>
+				<executions>
+					<execution>
+						<id>shade-flink</id>
+						<phase>package</phase>
+						<goals>
+							<goal>shade</goal>
+						</goals>
+						<configuration>
+							<artifactSet>
+								<includes>
+									<include>org.apache.flink:flink-connector-mongodb</include>
+									<include>org.mongodb:bson</include>
+									<include>org.mongodb:mongodb-driver-sync</include>
+									<include>org.mongodb:mongodb-driver-core</include>
+								</includes>
+							</artifactSet>
+							<filters>
+								<filter>
+									<artifact>org.mongodb:mongodb-driver-core</artifact>
+									<excludes>
+										<exclude>META-INF/native-image</exclude>

Review Comment:
   ```suggestion
   										<exclude>META-INF/native-image/**</exclude>
   ```



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsEqual.equalTo;

Review Comment:
   use assertj for all assertions



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsEqual.equalTo;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+@ExtendWith({TestLoggerExtension.class})
+public class MongoE2ECase extends TestLogger {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    public static final String MONGO_4_0 = "mongo:4.0.10";
+
+    private static final Network NETWORK = Network.newNetwork();
+
+    private final Path sqlConnectorMongoDBJar = ResourceTestUtils.getResource(".*mongodb.jar");
+
+    @Container
+    public static final MongoDBContainer MONGO_CONTAINER =

Review Comment:
   ```suggestion
       static final MongoDBContainer MONGO_CONTAINER =
   ```
   All Junit5 test class/methods/extensions should be package-private.



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsEqual.equalTo;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+@ExtendWith({TestLoggerExtension.class})
+public class MongoE2ECase extends TestLogger {

Review Comment:
   ```suggestion
   class MongoE2ECase {
   ```
   The `TestLogger` is junit4 infrastructure.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Serializable, Closeable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
+
+    private final MongoReadOptions readOptions;
+    private final boolean limitPushedDown;
+    private final MongoClient mongoClient;
+
+    public MongoSplitters(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            boolean limitPushedDown) {
+        this.readOptions = readOptions;
+        this.limitPushedDown = limitPushedDown;
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    public Collection<MongoScanSourceSplit> split(MongoNamespace namespace) {
+        BsonDocument collStats = MongoUtils.collStats(mongoClient, namespace);
+        if (!isCommandSucceed(collStats)) {
+            LOG.error(
+                    "Execute command collStats failed: {}",
+                    collStats.getString(ERROR_MESSAGE_FIELD));
+            throw new IllegalStateException(String.format("Collection not found %s", namespace));
+        }
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(readOptions, mongoClient, namespace, collStats);
+
+        if (limitPushedDown) {
+            LOG.info("Limit {} is applied, using single splitter", limitPushedDown);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);

Review Comment:
   I'd rather override the partition strategy (by setting `strategy`) in this branch and then handle it as usual below.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter implements MongoSplitters.MongoSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSampleSplitter.class);
+
+    public static final MongoSampleSplitter INSTANCE = new MongoSampleSplitter();
+
+    private MongoSampleSplitter() {}
+
+    @Override
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+
+        long count = splitContext.getCount();
+        long partitionSizeInBytes = readOptions.getPartitionSize().getBytes();
+        int samplesPerPartition = readOptions.getSamplesPerPartition();
+
+        long avgObjSizeInBytes = splitContext.getAvgObjSize();
+        long numDocumentsPerPartition = partitionSizeInBytes / avgObjSizeInBytes;
+
+        if (numDocumentsPerPartition >= count) {
+            LOG.info(
+                    "Fewer documents ({}) than the number of documents per partition ({}), fallback a SingleSplitter.",
+                    count,
+                    numDocumentsPerPartition);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        int numberOfSamples =
+                (int) Math.ceil((samplesPerPartition * count * 1.0d) / numDocumentsPerPartition);
+
+        List<BsonDocument> samples =
+                splitContext
+                        .getMongoCollection()
+                        .aggregate(
+                                Arrays.asList(
+                                        Aggregates.sample(numberOfSamples),
+                                        Aggregates.project(Projections.include(ID_FIELD)),
+                                        Aggregates.sort(Sorts.ascending(ID_FIELD))))
+                        .allowDiskUse(true)
+                        .into(new ArrayList<>());
+
+        List<MongoScanSourceSplit> sourceSplits = new ArrayList<>();
+        BsonDocument min = new BsonDocument(ID_FIELD, BSON_MIN_KEY);

Review Comment:
   ```suggestion
           BsonDocument partitionStart = new BsonDocument(ID_FIELD, BSON_MIN_KEY);
   ```



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/MongoTestUtil.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;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.model.Filters;
+import org.bson.Document;
+import org.slf4j.Logger;
+import org.testcontainers.containers.MongoDBContainer;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Collection of utility methods for MongoDB tests. */
+@Internal
+public class MongoTestUtil {
+
+    public static final String MONGO_4_0 = "mongo:4.0.10";
+
+    private MongoTestUtil() {}
+
+    /**
+     * Creates a preconfigured {@link MongoDBContainer}.
+     *
+     * @param dockerImageVersion describing the MongoDB image
+     * @param logger for test containers
+     * @return configured MongoDB container
+     */
+    public static MongoDBContainer createMongoDBContainer(
+            String dockerImageVersion, Logger logger) {
+        return new MongoDBContainer(DockerImageName.parse(dockerImageVersion))
+                .withLogConsumer(new Slf4jLogConsumer(logger));
+    }
+
+    public static void assertThatIdsAreNotWritten(MongoCollection<Document> coll, Integer... ids) {
+        boolean existOne = coll.find(Filters.in("_id", ids)).first() != null;
+        assertThat(existOne).isFalse();

Review Comment:
   Implement this just like assertThatIdsAreWritten but change the final assertion. This will result in a better error message, explicitly stating with id was found in the end.



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsEqual.equalTo;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+@ExtendWith({TestLoggerExtension.class})
+public class MongoE2ECase extends TestLogger {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    public static final String MONGO_4_0 = "mongo:4.0.10";
+
+    private static final Network NETWORK = Network.newNetwork();
+
+    private final Path sqlConnectorMongoDBJar = ResourceTestUtils.getResource(".*mongodb.jar");
+
+    @Container
+    public static final MongoDBContainer MONGO_CONTAINER =
+            new MongoDBContainer(MONGO_4_0)
+                    .withLogConsumer(new Slf4jLogConsumer(LOG))
+                    .withNetwork(NETWORK)
+                    .withNetworkAliases(MONGODB_HOSTNAME);
+
+    public static final TestcontainersSettings TESTCONTAINERS_SETTINGS =
+            TestcontainersSettings.builder()
+                    .logger(LOG)
+                    .network(NETWORK)
+                    .dependsOn(MONGO_CONTAINER)
+                    .build();
+
+    @RegisterExtension
+    public static final FlinkContainers FLINK =
+            FlinkContainers.builder()
+                    .withFlinkContainersSettings(
+                            FlinkContainersSettings.builder().numTaskManagers(2).build())
+                    .withTestcontainersSettings(TESTCONTAINERS_SETTINGS)
+                    .build();
+
+    private static MongoClient mongoClient;
+
+    @BeforeAll
+    private static void setUp() throws Exception {

Review Comment:
   ```suggestion
       static void setUp() throws Exception {
   ```
   Lifecycle methods must not be private (since junit 5.9.0).
   
   https://junit.org/junit5/docs/current/release-notes/#deprecations-and-breaking-changes-2



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+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.SINK_MAX_RETRIES;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int batchSize;
+    private final long batchIntervalMs;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    private MongoWriteOptions(
+            int batchSize,
+            long batchIntervalMs,
+            int maxRetries,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee) {
+        this.batchSize = batchSize;
+        this.batchIntervalMs = batchIntervalMs;
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+    }
+
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    public long getBatchIntervalMs() {
+        return batchIntervalMs;
+    }
+
+    public int getMaxRetries() {
+        return maxRetries;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return batchSize == that.batchSize
+                && batchIntervalMs == that.batchIntervalMs
+                && maxRetries == that.maxRetries
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                batchSize, batchIntervalMs, maxRetries, retryIntervalMs, deliveryGuarantee);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int batchSize = BUFFER_FLUSH_MAX_ROWS.defaultValue();
+        private long batchIntervalMs = BUFFER_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetries = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;

Review Comment:
   ```suggestion
           private DeliveryGuarantee deliveryGuarantee = DELIVERY_GUARANTEE.defaultValue();
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.DefaultMongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+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;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible for writing records to a MongoDB collection.
+ *
+ * @param <IN> The type of the input elements.
+ */
+@Internal
+public class MongoWriter<IN> implements SinkWriter<IN> {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoWriter.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+    private final MongoSinkContext sinkContext;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean flushOnCheckpoint;
+    private final List<WriteModel<BsonDocument>> bulkRequests = new ArrayList<>();
+    private final Collector<WriteModel<BsonDocument>> collector;
+    private final MongoClient mongoClient;
+
+    private boolean checkpointInProgress = false;
+    private volatile long lastSendTime = 0L;
+    private volatile long ackTime = Long.MAX_VALUE;
+
+    public MongoWriter(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            boolean flushOnCheckpoint,
+            Sink.InitContext initContext,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);
+        this.flushOnCheckpoint = flushOnCheckpoint;
+
+        checkNotNull(initContext);
+        this.mailboxExecutor = checkNotNull(initContext.getMailboxExecutor());
+
+        SinkWriterMetricGroup metricGroup = checkNotNull(initContext.metricGroup());
+        metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+
+        this.collector =
+                new CountingCollector<>(

Review Comment:
   Don't use this; it's not part of Flink's public API.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoShardedSplitter.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.MongoQueryException;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.UNAUTHORIZED_ERROR;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isValidShardedCollection;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readChunks;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readCollectionMetadata;
+
+/**
+ * Sharded Partitioner
+ *
+ * <p>Uses the chunks collection and partitions the collection based on the sharded collections
+ * chunk ranges.
+ *
+ * <p>The following config collections' read privilege is required.
+ *
+ * <ul>
+ *   <li>config.collections
+ *   <li>config.chunks
+ * </ul>
+ */
+@Internal
+public class MongoShardedSplitter implements MongoSplitters.MongoSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+    public static final MongoShardedSplitter INSTANCE = new MongoShardedSplitter();
+
+    private MongoShardedSplitter() {}
+
+    @Override
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoClient mongoClient = splitContext.getMongoClient();
+
+        List<BsonDocument> chunks;
+        BsonDocument collectionMetadata;
+        try {
+            collectionMetadata = readCollectionMetadata(mongoClient, namespace);
+            if (!isValidShardedCollection(collectionMetadata)) {
+                LOG.warn(
+                        "Collection {} does not appear to be sharded, fallback to SampleSplitter.",
+                        namespace);
+                return MongoSampleSplitter.INSTANCE.split(splitContext);
+            }
+            chunks = readChunks(mongoClient, collectionMetadata);
+        } catch (MongoQueryException e) {

Review Comment:
   Same question about errorHandling like in splitVector



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Serializable, Closeable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
+
+    private final MongoReadOptions readOptions;
+    private final boolean limitPushedDown;
+    private final MongoClient mongoClient;
+
+    public MongoSplitters(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            boolean limitPushedDown) {
+        this.readOptions = readOptions;
+        this.limitPushedDown = limitPushedDown;
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    public Collection<MongoScanSourceSplit> split(MongoNamespace namespace) {
+        BsonDocument collStats = MongoUtils.collStats(mongoClient, namespace);
+        if (!isCommandSucceed(collStats)) {
+            LOG.error(
+                    "Execute command collStats failed: {}",
+                    collStats.getString(ERROR_MESSAGE_FIELD));
+            throw new IllegalStateException(String.format("Collection not found %s", namespace));
+        }
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(readOptions, mongoClient, namespace, collStats);
+
+        if (limitPushedDown) {
+            LOG.info("Limit {} is applied, using single splitter", limitPushedDown);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        PartitionStrategy strategy = readOptions.getPartitionStrategy();
+        switch (strategy) {
+            case SINGLE:
+                return MongoSingleSplitter.INSTANCE.split(splitContext);
+            case SAMPLE:
+                return MongoSampleSplitter.INSTANCE.split(splitContext);
+            case SPLIT_VECTOR:
+                return MongoSplitVectorSplitter.INSTANCE.split(splitContext);
+            case SHARDED:
+                return MongoShardedSplitter.INSTANCE.split(splitContext);
+            case DEFAULT:
+            default:
+                return splitContext.isSharded()
+                        ? MongoShardedSplitter.INSTANCE.split(splitContext)
+                        : MongoSplitVectorSplitter.INSTANCE.split(splitContext);
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    /** Mongo Splitter to split a collection into multiple splits. */
+    public interface MongoSplitter {

Review Comment:
   With all implementations being singletons this interface doesn't need to exist. They could all just be static methods.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitContext.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt64;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARDED_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+
+/**
+ * The split context used by {@link MongoSplitters.MongoSplitter} to split collection into a set of
+ * chunks for MongoDB data source.
+ */
+@Internal
+public class MongoSplitContext {
+
+    /** Read options of MongoDB. */
+    private final MongoReadOptions readOptions;
+
+    /** Client of MongoDB. */
+    private final MongoClient mongoClient;
+
+    /** Namespace of MongoDB, eg. db.coll. */
+    private final MongoNamespace namespace;
+
+    /** Is a sharded collection. */
+    private final boolean sharded;
+
+    /** The number of objects or documents in this collection. */
+    private final long count;
+
+    /** The total uncompressed size(bytes) in memory of all records in a collection. */
+    private final long size;
+
+    /** The average size(bytes) of an object in the collection. */
+    private final long avgObjSize;
+
+    public MongoSplitContext(
+            MongoReadOptions readOptions,
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            boolean sharded,
+            long count,
+            long size,
+            long avgObjSize) {
+        this.readOptions = readOptions;
+        this.mongoClient = mongoClient;
+        this.namespace = namespace;
+        this.sharded = sharded;
+        this.count = count;
+        this.size = size;
+        this.avgObjSize = avgObjSize;
+    }
+
+    public static MongoSplitContext of(
+            MongoReadOptions readOptions,
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument collStats) {
+        return new MongoSplitContext(
+                readOptions,
+                mongoClient,
+                namespace,
+                collStats.getBoolean(SHARDED_FIELD, BsonBoolean.FALSE).getValue(),
+                collStats.getNumber(COUNT_FIELD, new BsonInt64(0)).longValue(),
+                collStats.getNumber(SIZE_FIELD, new BsonInt64(0)).longValue(),
+                collStats.getNumber(AVG_OBJ_SIZE_FIELD, new BsonInt64(0)).longValue());

Review Comment:
   Also, 0 is a dangerous default for AVG_OBJ_SIZE_FIELD since it'd result in divide-by-zero in the sample partitioner.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializer.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplitSerializer;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+
+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.List;
+import java.util.Map;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeList;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.deserializeMap;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeList;
+import static org.apache.flink.connector.mongodb.common.utils.MongoSerdeUtils.serializeMap;
+
+/** The {@link SimpleVersionedSerializer Serializer} for the enumerator state of Mongo source. */
+@Internal
+public class MongoSourceEnumStateSerializer
+        implements SimpleVersionedSerializer<MongoSourceEnumState> {
+
+    public static final MongoSourceEnumStateSerializer INSTANCE =
+            new MongoSourceEnumStateSerializer();
+
+    private MongoSourceEnumStateSerializer() {
+        // Singleton instance.
+    }
+
+    @Override
+    public int getVersion() {
+        // We use MongoSourceSplitSerializer's version because we use reuse this class.
+        return MongoSourceSplitSerializer.CURRENT_VERSION;
+    }
+
+    @Override
+    public byte[] serialize(MongoSourceEnumState state) throws IOException {
+        try (ByteArrayOutputStream baos = new ByteArrayOutputStream();
+                DataOutputStream out = new DataOutputStream(baos)) {
+            serializeList(out, state.getRemainingCollections(), DataOutputStream::writeUTF);
+
+            serializeList(out, state.getAlreadyProcessedCollections(), DataOutputStream::writeUTF);
+
+            serializeList(
+                    out,
+                    state.getRemainingScanSplits(),
+                    MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
+
+            serializeMap(
+                    out,
+                    state.getAssignedScanSplits(),
+                    DataOutputStream::writeUTF,
+                    MongoSourceSplitSerializer.INSTANCE::serializeMongoSplit);
+
+            out.writeBoolean(state.isInitialized());
+
+            out.flush();
+            return baos.toByteArray();
+        }
+    }
+
+    @Override
+    public MongoSourceEnumState deserialize(int version, byte[] serialized) throws IOException {
+        try (ByteArrayInputStream bais = new ByteArrayInputStream(serialized);
+                DataInputStream in = new DataInputStream(bais)) {
+            List<String> remainingCollections = deserializeList(in, DataInput::readUTF);
+            List<String> alreadyProcessedCollections = deserializeList(in, DataInput::readUTF);
+            List<MongoScanSourceSplit> remainingScanSplits =
+                    deserializeList(in, i -> deserializeMongoScanSourceSplit(version, i));
+
+            Map<String, MongoScanSourceSplit> assignedScanSplits =
+                    deserializeMap(
+                            in,
+                            DataInput::readUTF,
+                            i -> deserializeMongoScanSourceSplit(version, i));
+
+            boolean initialized = in.readBoolean();
+
+            return new MongoSourceEnumState(
+                    remainingCollections,
+                    alreadyProcessedCollections,
+                    remainingScanSplits,
+                    assignedScanSplits,
+                    initialized);
+        }
+    }
+
+    private MongoScanSourceSplit deserializeMongoScanSourceSplit(int version, DataInputStream in)
+            throws IOException {
+        return (MongoScanSourceSplit)

Review Comment:
   I'd try to add split serializer parameter to the MongoSource constructor that is then also passed into the enum state serializer.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable
+    public static BsonDocument readCollectionMetadata(
+            MongoClient mongoClient, MongoNamespace namespace) {
+        MongoCollection<BsonDocument> collection =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(COLLECTIONS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        return collection
+                .find(eq(ID_FIELD, namespace.getFullName()))
+                .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD))

Review Comment:
   What is this `DROPPED_FIELD` about? Is this about whether the `collection` was dropped?



##########
flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java:
##########
@@ -0,0 +1,158 @@
+/*
+ * 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.tests.util.mongodb;
+
+import org.apache.flink.api.common.time.Deadline;
+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.test.resources.ResourceTestUtils;
+import org.apache.flink.test.util.SQLJobSubmission;
+import org.apache.flink.util.TestLogger;
+import org.apache.flink.util.TestLoggerExtension;
+
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.MongoDatabase;
+import org.bson.Document;
+import org.bson.types.ObjectId;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+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 org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.junit.jupiter.Container;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsEqual.equalTo;
+
+/** End-to-end test for the MongoDB connectors. */
+@Testcontainers
+@ExtendWith({TestLoggerExtension.class})
+public class MongoE2ECase extends TestLogger {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoE2ECase.class);
+
+    private static final String MONGODB_HOSTNAME = "mongodb";
+
+    public static final String MONGO_4_0 = "mongo:4.0.10";

Review Comment:
   This constant should be deduplicated with the `MongoTestUtil`.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue() == 1;
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue() == 1L;
+        }
+        if (bsonValue.isString()) {
+            return Boolean.parseBoolean(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to boolean from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static byte convertToTinyInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (byte) 1 : (byte) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (byte) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (byte) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Byte.parseByte(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to tinyint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static short convertToSmallInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (short) 1 : (short) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (short) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (short) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Short.parseShort(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to smallint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static int convertToInt(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().intValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.intValue();
+            } else if (decimal128Value.isNegative()) {
+                return Integer.MIN_VALUE;
+            } else {
+                return Integer.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isDateTime()) {
+            return (int) Instant.ofEpochMilli(bsonValue.asDateTime().getValue()).getEpochSecond();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime();
+        }
+        if (bsonValue.isString()) {
+            return Integer.parseInt(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to integer from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static long convertToLong(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().longValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.longValue();
+            } else if (decimal128Value.isNegative()) {
+                return Long.MIN_VALUE;
+            } else {
+                return Long.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1L : 0L;
+        }
+        if (bsonValue.isDateTime()) {
+            return bsonValue.asDateTime().getValue();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime() * 1000L;
+        }
+        if (bsonValue.isString()) {
+            return Long.parseLong(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to long from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static double convertToDouble(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().doubleValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.doubleValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Double.MAX_VALUE;
+            } else {
+                return Double.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isString()) {
+            return Double.parseDouble(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to double from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static float convertToFloat(BsonValue bsonValue) {
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {

Review Comment:
   I certainly see where you're coming from.
   
   However,
   
   * for consistency with other connectors we should eagerly reject types that can't be properly represented in Flink, like objectId, JavascriptCode etc. There is definitely a need for having connector-specific types, but until that is implemented we should restrict us to types that "fit" right now. This will ensure we don't run into problems once we can properly support these types (because we'd change the behavior of the connector w.r.t. how it maps data types).
   * the JSON format eagerly rejects cases where the type for a given field differs between 2 records. We should follow the same behavior, and then consider looking at this issue holistically across all formats. In general schema-less systems aren't quite fitting into Flink's type system yet.
   
   We have a real problem with connectors behaving inconsistently right now; let's try to keep that to a minimum (even if we that means added limitations that we aren't happy with) :/



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitContext.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt64;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARDED_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+
+/**
+ * The split context used by {@link MongoSplitters.MongoSplitter} to split collection into a set of
+ * chunks for MongoDB data source.
+ */
+@Internal
+public class MongoSplitContext {
+
+    /** Read options of MongoDB. */
+    private final MongoReadOptions readOptions;
+
+    /** Client of MongoDB. */
+    private final MongoClient mongoClient;
+
+    /** Namespace of MongoDB, eg. db.coll. */
+    private final MongoNamespace namespace;
+
+    /** Is a sharded collection. */
+    private final boolean sharded;
+
+    /** The number of objects or documents in this collection. */
+    private final long count;
+
+    /** The total uncompressed size(bytes) in memory of all records in a collection. */
+    private final long size;
+
+    /** The average size(bytes) of an object in the collection. */
+    private final long avgObjSize;
+
+    public MongoSplitContext(
+            MongoReadOptions readOptions,
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            boolean sharded,
+            long count,
+            long size,
+            long avgObjSize) {
+        this.readOptions = readOptions;
+        this.mongoClient = mongoClient;
+        this.namespace = namespace;
+        this.sharded = sharded;
+        this.count = count;
+        this.size = size;
+        this.avgObjSize = avgObjSize;
+    }
+
+    public static MongoSplitContext of(
+            MongoReadOptions readOptions,
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument collStats) {
+        return new MongoSplitContext(
+                readOptions,
+                mongoClient,
+                namespace,
+                collStats.getBoolean(SHARDED_FIELD, BsonBoolean.FALSE).getValue(),
+                collStats.getNumber(COUNT_FIELD, new BsonInt64(0)).longValue(),
+                collStats.getNumber(SIZE_FIELD, new BsonInt64(0)).longValue(),
+                collStats.getNumber(AVG_OBJ_SIZE_FIELD, new BsonInt64(0)).longValue());

Review Comment:
   Why do we need these defaults?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";

Review Comment:
   these can all be final



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;

Review Comment:
   move to MongoConstants



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable
+    public static BsonDocument readCollectionMetadata(
+            MongoClient mongoClient, MongoNamespace namespace) {
+        MongoCollection<BsonDocument> collection =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(COLLECTIONS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        return collection
+                .find(eq(ID_FIELD, namespace.getFullName()))
+                .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD))
+                .first();
+    }
+
+    public static boolean isValidShardedCollection(BsonDocument collectionMetadata) {
+        return collectionMetadata != null
+                && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue();
+    }

Review Comment:
   How does the `dropped` field relate to sharding?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoJsonDeserializationSchema.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.reader.deserializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+
+import org.bson.BsonDocument;
+import org.bson.json.JsonMode;
+
+import java.util.Optional;
+
+/**
+ * A schema bridge for deserializing the MongoDB's {@code BsonDocument} to MongoDB's {@link
+ * JsonMode}'s RELAXED Json string.
+ */
+@PublicEvolving
+public class MongoJsonDeserializationSchema implements MongoDeserializationSchema<String> {

Review Comment:
   What's the use-case for this schema?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Serializable, Closeable {

Review Comment:
   Why is this serializable? It's shouldn't be required, and with the non-serializable MongoClient shouldn't work anyway.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/PartitionStrategy.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+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;
+
+/** MongoSplitStrategy that can be chosen. */
+@PublicEvolving
+public enum PartitionStrategy implements DescribedEnum {

Review Comment:
   Can you give me some pros/cons for the different strategies?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitVectorSplitter.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoCommandException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.apache.commons.collections.CollectionUtils;
+import org.bson.BsonArray;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SPLIT_KEYS_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.UNAUTHORIZED_ERROR;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.splitVector;
+
+/**
+ * SplitVector Partitioner
+ *
+ * <p>Uses the SplitVector command to generate chunks for a collection. eg. <code>
+ * db.runCommand({splitVector:"inventory.products", keyPattern:{_id:1}, maxChunkSize:64})</code>
+ *
+ * <p>Requires splitVector privilege.
+ */
+@Internal
+public class MongoSplitVectorSplitter implements MongoSplitters.MongoSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitVectorSplitter.class);
+
+    public static final MongoSplitVectorSplitter INSTANCE = new MongoSplitVectorSplitter();
+
+    private MongoSplitVectorSplitter() {}
+
+    @Override
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoClient mongoClient = splitContext.getMongoClient();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+
+        MemorySize chunkSize = readOptions.getPartitionSize();
+        // if partition size < 1mb, use 1 mb as chunk size.
+        int maxChunkSizeMB = Math.max(chunkSize.getMebiBytes(), 1);
+
+        BsonDocument keyPattern = new BsonDocument(ID_FIELD, new BsonInt32(1));
+
+        BsonDocument splitResult;
+        try {
+            splitResult = splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB);
+        } catch (MongoCommandException e) {
+            if (e.getErrorCode() == UNAUTHORIZED_ERROR) {
+                LOG.warn(
+                        "Unauthorized to execute splitVector command: {}, fallback to SampleSplitter",
+                        e.getErrorMessage());
+            } else {
+                LOG.warn(
+                        "Execute splitVector command failed: {}, fallback to SampleSplitter",
+                        e.getErrorMessage());

Review Comment:
   We should fail hard in this case. The user told us to use splitVector, presumably for a good reason, so have them decide how to handle this issue.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.

Review Comment:
   > The number of samples to take per partition
   
   This doesn't make sense to me.
   If multiple samples are taken per partition then _somewhere_ in here we'd have to merge sample to arrive at a single partition again, but afaict that doesn't happen.
   
   Instead we have some strange formula that determines the number of samples (read: partitions), and I have no idea how the resulting partitions could correlate with the desired partition size.
   
   Why isnt the number of sample (again: partitions) not `count / numDocumentsPerPartition`?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Serializable, Closeable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
+
+    private final MongoReadOptions readOptions;
+    private final boolean limitPushedDown;
+    private final MongoClient mongoClient;
+
+    public MongoSplitters(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            boolean limitPushedDown) {
+        this.readOptions = readOptions;
+        this.limitPushedDown = limitPushedDown;
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    public Collection<MongoScanSourceSplit> split(MongoNamespace namespace) {
+        BsonDocument collStats = MongoUtils.collStats(mongoClient, namespace);
+        if (!isCommandSucceed(collStats)) {
+            LOG.error(
+                    "Execute command collStats failed: {}",
+                    collStats.getString(ERROR_MESSAGE_FIELD));
+            throw new IllegalStateException(String.format("Collection not found %s", namespace));
+        }
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(readOptions, mongoClient, namespace, collStats);
+
+        if (limitPushedDown) {
+            LOG.info("Limit {} is applied, using single splitter", limitPushedDown);

Review Comment:
   ```suggestion
               LOG.info("Limit {} is applied, using single split partition strategy.", limitPushedDown);
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/deserializer/MongoDeserializationSchema.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.deserializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+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.
+ *
+ * @param <T> The output message type for sinking to downstream flink operator.
+ */
+@PublicEvolving
+public interface MongoDeserializationSchema<T> extends Serializable, ResultTypeQueryable<T> {
+
+    /**
+     * 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.
+     *
+     * <p>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 out The collector to put the resulting messages.
+     */
+    @PublicEvolving

Review Comment:
   ```suggestion
   ```



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSampleSplitter.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.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.Projections;
+import com.mongodb.client.model.Sorts;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+
+/**
+ * Sample Partitioner
+ *
+ * <p>Samples the collection to generate partitions.
+ *
+ * <p>Uses the average document size to split the collection into average sized chunks
+ *
+ * <p>The partitioner samples the collection, projects and sorts by the partition fields. Then uses
+ * every {@code samplesPerPartition} as the value to use to calculate the partition boundaries.
+ *
+ * <ul>
+ *   <li>scan.partition.size: The average size (MB) for each partition. Note: Uses the average
+ *       document size to determine the number of documents per partition so may not be even.
+ *       Defaults to: 64mb.
+ *   <li>scan.partition.samples: The number of samples to take per partition. Defaults to: 10. The
+ *       total number of samples taken is calculated as: {@code samples per partition * (count of
+ *       documents / number of documents per partition)}.
+ * </ul>
+ */
+@Internal
+public class MongoSampleSplitter implements MongoSplitters.MongoSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSampleSplitter.class);
+
+    public static final MongoSampleSplitter INSTANCE = new MongoSampleSplitter();
+
+    private MongoSampleSplitter() {}
+
+    @Override
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+
+        long count = splitContext.getCount();
+        long partitionSizeInBytes = readOptions.getPartitionSize().getBytes();
+        int samplesPerPartition = readOptions.getSamplesPerPartition();
+
+        long avgObjSizeInBytes = splitContext.getAvgObjSize();
+        long numDocumentsPerPartition = partitionSizeInBytes / avgObjSizeInBytes;
+
+        if (numDocumentsPerPartition >= count) {
+            LOG.info(
+                    "Fewer documents ({}) than the number of documents per partition ({}), fallback a SingleSplitter.",
+                    count,
+                    numDocumentsPerPartition);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }
+
+        int numberOfSamples =
+                (int) Math.ceil((samplesPerPartition * count * 1.0d) / numDocumentsPerPartition);
+
+        List<BsonDocument> samples =

Review Comment:
   I find it very confusing that this is named "samples"; I would expect this to be used to derive _some_ quality of the whole dataset (like `avgObjSizeInBytes`), but these are rather pointers into the key space / partition boundaries.



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitVectorSplitter.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoCommandException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.apache.commons.collections.CollectionUtils;
+import org.bson.BsonArray;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SPLIT_KEYS_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.UNAUTHORIZED_ERROR;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.splitVector;
+
+/**
+ * SplitVector Partitioner
+ *
+ * <p>Uses the SplitVector command to generate chunks for a collection. eg. <code>
+ * db.runCommand({splitVector:"inventory.products", keyPattern:{_id:1}, maxChunkSize:64})</code>
+ *
+ * <p>Requires splitVector privilege.
+ */
+@Internal
+public class MongoSplitVectorSplitter implements MongoSplitters.MongoSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitVectorSplitter.class);
+
+    public static final MongoSplitVectorSplitter INSTANCE = new MongoSplitVectorSplitter();
+
+    private MongoSplitVectorSplitter() {}
+
+    @Override
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoClient mongoClient = splitContext.getMongoClient();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+
+        MemorySize chunkSize = readOptions.getPartitionSize();
+        // if partition size < 1mb, use 1 mb as chunk size.
+        int maxChunkSizeMB = Math.max(chunkSize.getMebiBytes(), 1);
+
+        BsonDocument keyPattern = new BsonDocument(ID_FIELD, new BsonInt32(1));
+
+        BsonDocument splitResult;
+        try {
+            splitResult = splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB);

Review Comment:
   Why does this sometimes throw an exception and other times encodes an error in the split result?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoShardedSplitter.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.MongoQueryException;
+import com.mongodb.client.MongoClient;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.UNAUTHORIZED_ERROR;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isValidShardedCollection;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readChunks;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.readCollectionMetadata;
+
+/**
+ * Sharded Partitioner
+ *
+ * <p>Uses the chunks collection and partitions the collection based on the sharded collections
+ * chunk ranges.
+ *
+ * <p>The following config collections' read privilege is required.
+ *
+ * <ul>
+ *   <li>config.collections
+ *   <li>config.chunks
+ * </ul>
+ */
+@Internal
+public class MongoShardedSplitter implements MongoSplitters.MongoSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+    public static final MongoShardedSplitter INSTANCE = new MongoShardedSplitter();
+
+    private MongoShardedSplitter() {}
+
+    @Override
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoClient mongoClient = splitContext.getMongoClient();
+
+        List<BsonDocument> chunks;
+        BsonDocument collectionMetadata;
+        try {
+            collectionMetadata = readCollectionMetadata(mongoClient, namespace);
+            if (!isValidShardedCollection(collectionMetadata)) {
+                LOG.warn(
+                        "Collection {} does not appear to be sharded, fallback to SampleSplitter.",

Review Comment:
   see splitVector comments



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol commented on PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#issuecomment-1422714527

   I pushed a few commits to get the PR into a mergable state.
   
   1) I removed the mockito dependency from the sample splitter test. Mockito usage is heavily discouraged (see Flink code contribution guide), and instead I slightly refactored the code to make this work.
   2) Splitter "implementations" are now just static methods since singletons don't provide a benefit.
   3) TestLoggerExtension service entry was removed, because it is implicitly activated by pulling in flink-test-utils-junit.
   
   4) Refactored the MongoSampleSplitterTest and fixed some issues with the sampling.
   
   The current test was too much of a re-implementation of the internal logic, which was highlighted by it not detecting problems in how the sampling and merging was done.
   
   The current code used `N = numPartitions * numSamplesPerPartition` samples. Seems correct on first glance, but these `N` samples actually split the input into `N + 1` partitions.
   
   ```
   Samples:           0 1 2 3   (4)
   Implicit bounds: -         + (min/max)
   Partitions:      |-|-|-|-|-| (5)
   ```
   
   This resulted in an uneven distribution of data across splits.
   
   For example, if we want to create 2 partitions, with 2 samples per partition, then we'd like to get this:
   
   (- == Min ; + == Max ; x == Sample #x)
   
   ```
   Samples:    - 0 1 2 +
   Partitions: |---|---|
   ```
   
   Meanwhile, the current code did this:
   ```
   
   Samples:    - 1 2 +
   Partitions: |-|---|
   ```
   
   The replacement of the first/last sampling with Min/Max was removed because it made it very difficult to reason about the code, as we actually changed the number of partitions.
   
   Let's say we want 2 partitions with 2 samples each => 4 samples.
   
   Before replacement:
   
   ```
   Samples:           0 1 2 3   (4)
   Implicit bounds: -         + (min/max)
   Partitions:      |-|-|-|-|-| (5)
   ```
   
   
   After replacement:
   ```
   
   Samples:             1 2     (2)
   Implicit bounds: -         + (min/max)
   Partitions:      |---|-|---| (3)
   ```
   
   
   Instead we now ask for N-1 samples, and use min/max as additional points:
   
   ```
   Samples:           0 1 2   (3)
   Implicit bounds: -         + (min/max)
   Partitions:      |-|-|-|-| (4)
   Splits:          |---|---|
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025508153


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable
+    public static BsonDocument readCollectionMetadata(
+            MongoClient mongoClient, MongoNamespace namespace) {
+        MongoCollection<BsonDocument> collection =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(COLLECTIONS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        return collection
+                .find(eq(ID_FIELD, namespace.getFullName()))
+                .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD))
+                .first();
+    }
+
+    public static boolean isValidShardedCollection(BsonDocument collectionMetadata) {
+        return collectionMetadata != null
+                && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue();
+    }
+
+    public static List<BsonDocument> readChunks(
+            MongoClient mongoClient, BsonDocument collectionMetadata) {
+        MongoCollection<BsonDocument> chunks =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(CHUNKS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        Bson filter =
+                or(
+                        new BsonDocument(NAMESPACE_FIELD, collectionMetadata.get(ID_FIELD)),
+                        // MongoDB 4.9.0 removed ns field of config.chunks collection, using

Review Comment:
   MongoDB's java driver has good compatibility.
   Using the latest driver can theoretically adapt to any MongoDB version.
   
   https://www.mongodb.com/docs/drivers/java/sync/current/compatibility
   
   MongoDB 6.0 was released recently. I think versions 4.x and 5.x are currently widely used.
   We intend to support versions after MongoDB 3.6.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025492765


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * 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.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+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.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;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+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. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable
+    public static BsonDocument readCollectionMetadata(
+            MongoClient mongoClient, MongoNamespace namespace) {
+        MongoCollection<BsonDocument> collection =
+                mongoClient
+                        .getDatabase(CONFIG_DATABASE)
+                        .getCollection(COLLECTIONS_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        return collection
+                .find(eq(ID_FIELD, namespace.getFullName()))
+                .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD))
+                .first();
+    }
+
+    public static boolean isValidShardedCollection(BsonDocument collectionMetadata) {
+        return collectionMetadata != null
+                && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue();
+    }

Review Comment:
   The `collectionMetadata` was got from `config.collections`.
   The [config.collections](https://www.mongodb.com/docs/v5.0/reference/config-database/#mongodb-data-config.collections) stores a document for each sharded (only sharded) collection in the cluster.
   
   As commented above, we need to filter out sharded collections that have been dropped.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1025564871


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Serializable, Closeable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
+
+    private final MongoReadOptions readOptions;
+    private final boolean limitPushedDown;
+    private final MongoClient mongoClient;
+
+    public MongoSplitters(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            boolean limitPushedDown) {
+        this.readOptions = readOptions;
+        this.limitPushedDown = limitPushedDown;
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    public Collection<MongoScanSourceSplit> split(MongoNamespace namespace) {
+        BsonDocument collStats = MongoUtils.collStats(mongoClient, namespace);
+        if (!isCommandSucceed(collStats)) {
+            LOG.error(
+                    "Execute command collStats failed: {}",
+                    collStats.getString(ERROR_MESSAGE_FIELD));
+            throw new IllegalStateException(String.format("Collection not found %s", namespace));
+        }
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(readOptions, mongoClient, namespace, collStats);
+
+        if (limitPushedDown) {
+            LOG.info("Limit {} is applied, using single splitter", limitPushedDown);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }

Review Comment:
   Another reason for why we don't try to split things is that it is difficult for us to merge the results of each chunk when reading concurrently. We need to cache these results to sort and limit, which may cause a lot of overhead.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1027761305


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+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.SINK_MAX_RETRIES;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.SINK_RETRY_INTERVAL;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int batchSize;
+    private final long batchIntervalMs;
+    private final int maxRetries;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+
+    private MongoWriteOptions(
+            int batchSize,
+            long batchIntervalMs,
+            int maxRetries,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee) {
+        this.batchSize = batchSize;
+        this.batchIntervalMs = batchIntervalMs;
+        this.maxRetries = maxRetries;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+    }
+
+    public int getBatchSize() {
+        return batchSize;
+    }
+
+    public long getBatchIntervalMs() {
+        return batchIntervalMs;
+    }
+
+    public int getMaxRetries() {
+        return maxRetries;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return batchSize == that.batchSize
+                && batchIntervalMs == that.batchIntervalMs
+                && maxRetries == that.maxRetries
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                batchSize, batchIntervalMs, maxRetries, retryIntervalMs, deliveryGuarantee);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int batchSize = BUFFER_FLUSH_MAX_ROWS.defaultValue();
+        private long batchIntervalMs = BUFFER_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetries = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param batchSize the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBatchSize(int batchSize) {
+            checkArgument(
+                    batchSize == -1 || batchSize > 0,
+                    "Max number of batch size must be larger than 0.");
+            this.batchSize = batchSize;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param batchIntervalMs the batch flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBatchIntervalMs(long batchIntervalMs) {
+            checkArgument(
+                    batchIntervalMs == -1 || batchIntervalMs >= 0,
+                    "The batch flush interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");

Review Comment:
   There are currently three ways to write:
   1.  flush only on checkpoint
   2. flush for for every single element
   3. flush when time interval or  batch size exceed limi
   
   | batch interval \\ batch size  |  -1  |  1  | > 1  |
   |--|--|--|--|
   | -1  |  flush on checkpoint  |  flush for every single element | flush when batch size exceed limit |
   | 0  |  flush for every single element  |  flush for every single element  | flush for every single element |
   | > 0  |  flush when time interval exceed limit  | flush for every single element | flush when time interval or  batch size exceed limit |



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1028370047


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitVectorSplitter.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoCommandException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.apache.commons.collections.CollectionUtils;
+import org.bson.BsonArray;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SPLIT_KEYS_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.UNAUTHORIZED_ERROR;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.splitVector;
+
+/**
+ * SplitVector Partitioner
+ *
+ * <p>Uses the SplitVector command to generate chunks for a collection. eg. <code>
+ * db.runCommand({splitVector:"inventory.products", keyPattern:{_id:1}, maxChunkSize:64})</code>
+ *
+ * <p>Requires splitVector privilege.
+ */
+@Internal
+public class MongoSplitVectorSplitter implements MongoSplitters.MongoSplitter {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitVectorSplitter.class);
+
+    public static final MongoSplitVectorSplitter INSTANCE = new MongoSplitVectorSplitter();
+
+    private MongoSplitVectorSplitter() {}
+
+    @Override
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext splitContext) {
+        MongoClient mongoClient = splitContext.getMongoClient();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+
+        MemorySize chunkSize = readOptions.getPartitionSize();
+        // if partition size < 1mb, use 1 mb as chunk size.
+        int maxChunkSizeMB = Math.max(chunkSize.getMebiBytes(), 1);
+
+        BsonDocument keyPattern = new BsonDocument(ID_FIELD, new BsonInt32(1));
+
+        BsonDocument splitResult;
+        try {
+            splitResult = splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB);

Review Comment:
   Thanks @zentol, I checked driver source code of `InternalStreamConnection#receiveCommandMessageResponse`. If the execution of the command fails, an exception will be thrown, and we don't need to deal with other case.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1028927993


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue() == 1;
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue() == 1L;
+        }
+        if (bsonValue.isString()) {
+            return Boolean.parseBoolean(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to boolean from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static byte convertToTinyInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (byte) 1 : (byte) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (byte) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (byte) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Byte.parseByte(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to tinyint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static short convertToSmallInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (short) 1 : (short) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (short) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (short) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Short.parseShort(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to smallint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static int convertToInt(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().intValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.intValue();
+            } else if (decimal128Value.isNegative()) {
+                return Integer.MIN_VALUE;
+            } else {
+                return Integer.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isDateTime()) {
+            return (int) Instant.ofEpochMilli(bsonValue.asDateTime().getValue()).getEpochSecond();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime();
+        }
+        if (bsonValue.isString()) {
+            return Integer.parseInt(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to integer from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static long convertToLong(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().longValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.longValue();
+            } else if (decimal128Value.isNegative()) {
+                return Long.MIN_VALUE;
+            } else {
+                return Long.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1L : 0L;
+        }
+        if (bsonValue.isDateTime()) {
+            return bsonValue.asDateTime().getValue();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime() * 1000L;
+        }
+        if (bsonValue.isString()) {
+            return Long.parseLong(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to long from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static double convertToDouble(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().doubleValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.doubleValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Double.MAX_VALUE;
+            } else {
+                return Double.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isString()) {
+            return Double.parseDouble(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to double from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static float convertToFloat(BsonValue bsonValue) {
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {

Review Comment:
   Thanks @zentol for the suggestion.
   
   I'm ok with 1:1 type mapping and don't support these special types at the moment such as `BsonSymbol`,`BsonRegularExpression`, `BsonJavaScript`, `BsonDbPointer`.
   
   But we have to face the fact that [ObjectId](https://www.mongodb.com/docs/manual/reference/method/ObjectId/#objectid) is used as the default primary key data type of MongoDB. When the user inserts data and other types of primary key are not explicitly used, MongoDB will automatically generate an ObjectId as the primary key of this document. I'm sure this kind of scenario is very common in the use of MongoDB. If we directly reject the conversion of ObjectId in MongoDB, it will not be available in many cases.
   
   A workaround is to only provide conversion of these special types to string ([MongoDB extended json format](https://www.mongodb.com/docs/drivers/java/sync/v4.3/fundamentals/data-formats/document-data-format-extended-json/#document-data-format--extended-json)), and let user decide how to convert it by UDF. Do you think we can do it this way?
   
   https://www.mongodb.com/docs/manual/reference/mongodb-extended-json/
   
   Example Field Name | Canonical Format | Relaxed Format
   -- | -- | --
   "_id:" | {"$oid":"5d505646cf6d4fe581014ab2"} | {"$oid":"5d505646cf6d4fe581014ab2"}
   "arrayField": | ["hello",{"$numberInt":"10"}] | ["hello",10]
   "dateField": | {"$date":{"$numberLong":"1565546054692"}} | {"$date":"2019-08-11T17:54:14.692Z"}
   "dateBefore1970": | {"$date":{"$numberLong":"-1577923200000"}} | {"$date":{"$numberLong":"-1577923200000"}}
   "decimal128Field": | {"$numberDecimal":"10.99"} | {"$numberDecimal":"10.99"}
   "documentField": | {"a":"hello"} | {"a":"hello"}
   "doubleField": | {"$numberDouble":"10.5"} | 10.5
   "infiniteNumber" | {"$numberDouble":"Infinity"} | {"$numberDouble":"Infinity"}
   "int32field": | {"$numberInt":"10"} | 10
   "int64Field": | {"$numberLong":"50"} | 50
   "minKeyField": | {"$minKey":1} | {"$minKey":1}
   "maxKeyField": | {"$maxKey":1} | {"$maxKey":1}
   "regexField": | {"$regularExpression":{"pattern":"^H","options":"i"}} | {"$regularExpression":{"pattern":"^H","options":"i"}}
   "timestampField": | {"$timestamp":{"t":1565545664,"i":1}} | {"$timestamp":{"t":1565545664,"i":1}}
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1035106483


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitContext.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt64;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.AVG_OBJ_SIZE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.COUNT_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARDED_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.SIZE_FIELD;
+
+/**
+ * The split context used by {@link MongoSplitters.MongoSplitter} to split collection into a set of
+ * chunks for MongoDB data source.
+ */
+@Internal
+public class MongoSplitContext {
+
+    /** Read options of MongoDB. */
+    private final MongoReadOptions readOptions;
+
+    /** Client of MongoDB. */
+    private final MongoClient mongoClient;
+
+    /** Namespace of MongoDB, eg. db.coll. */
+    private final MongoNamespace namespace;
+
+    /** Is a sharded collection. */
+    private final boolean sharded;
+
+    /** The number of objects or documents in this collection. */
+    private final long count;
+
+    /** The total uncompressed size(bytes) in memory of all records in a collection. */
+    private final long size;
+
+    /** The average size(bytes) of an object in the collection. */
+    private final long avgObjSize;
+
+    public MongoSplitContext(
+            MongoReadOptions readOptions,
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            boolean sharded,
+            long count,
+            long size,
+            long avgObjSize) {
+        this.readOptions = readOptions;
+        this.mongoClient = mongoClient;
+        this.namespace = namespace;
+        this.sharded = sharded;
+        this.count = count;
+        this.size = size;
+        this.avgObjSize = avgObjSize;
+    }
+
+    public static MongoSplitContext of(
+            MongoReadOptions readOptions,
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument collStats) {
+        return new MongoSplitContext(
+                readOptions,
+                mongoClient,
+                namespace,
+                collStats.getBoolean(SHARDED_FIELD, BsonBoolean.FALSE).getValue(),
+                collStats.getNumber(COUNT_FIELD, new BsonInt64(0)).longValue(),
+                collStats.getNumber(SIZE_FIELD, new BsonInt64(0)).longValue(),
+                collStats.getNumber(AVG_OBJ_SIZE_FIELD, new BsonInt64(0)).longValue());

Review Comment:
   When the collection is empty, `avgObjSize` field will not be included in `collStats`. 
   We should add some validation to avoid division by zero in the sample partitioner.
   
   ```json
   {
       "ns": "test_source.empty_coll",
       "size": 0, 
       "count": 0, 
       "storageSize": 4096,
       "capped": false
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1035041422


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitters.java:
##########
@@ -0,0 +1,110 @@
+/*
+ * 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.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.common.utils.MongoUtils;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+import org.apache.flink.connector.mongodb.source.split.MongoSourceSplit;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Collection;
+
+import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+
+/** To split collections of MongoDB to {@link MongoSourceSplit}s. */
+@Internal
+public class MongoSplitters implements Serializable, Closeable {
+
+    private static final long serialVersionUID = 1L;
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoSplitters.class);
+
+    private final MongoReadOptions readOptions;
+    private final boolean limitPushedDown;
+    private final MongoClient mongoClient;
+
+    public MongoSplitters(
+            MongoConnectionOptions connectionOptions,
+            MongoReadOptions readOptions,
+            boolean limitPushedDown) {
+        this.readOptions = readOptions;
+        this.limitPushedDown = limitPushedDown;
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    public Collection<MongoScanSourceSplit> split(MongoNamespace namespace) {
+        BsonDocument collStats = MongoUtils.collStats(mongoClient, namespace);
+        if (!isCommandSucceed(collStats)) {
+            LOG.error(
+                    "Execute command collStats failed: {}",
+                    collStats.getString(ERROR_MESSAGE_FIELD));
+            throw new IllegalStateException(String.format("Collection not found %s", namespace));
+        }
+
+        MongoSplitContext splitContext =
+                MongoSplitContext.of(readOptions, mongoClient, namespace, collStats);
+
+        if (limitPushedDown) {
+            LOG.info("Limit {} is applied, using single splitter", limitPushedDown);
+            return MongoSingleSplitter.INSTANCE.split(splitContext);
+        }

Review Comment:
   @zentol 
   I made some test for that case, the `Table` runtime indeed deal with the rest but the `DataStream` runtime seems not. 
   Is it necessary to allow limits under the `DataStream` API as well ?
   If necessary, where would it be appropriate to deal with the rest, maybe `RecordEmitter` ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#issuecomment-1278767321

   @Jiabao-Sun Please be aware that the review may take a while; I may not even get to it next week.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1016847978


##########
pom.xml:
##########
@@ -0,0 +1,426 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+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.
+-->
+<project
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+	xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<parent>
+		<groupId>io.github.zentol.flink</groupId>
+		<artifactId>flink-connector-parent</artifactId>
+		<version>1.0</version>
+	</parent>
+
+	<groupId>org.apache.flink</groupId>
+	<artifactId>flink-connector-mongodb-parent</artifactId>
+	<version>1.0-SNAPSHOT</version>
+
+	<name>Flink : Connectors : MongoDB Parent</name>
+	<packaging>pom</packaging>
+	<url>https://flink.apache.org</url>
+	<inceptionYear>2022</inceptionYear>
+
+	<licenses>
+		<license>
+			<name>The Apache Software License, Version 2.0</name>
+			<url>https://www.apache.org/licenses/LICENSE-2.0.txt</url>
+			<distribution>repo</distribution>
+		</license>
+	</licenses>
+
+	<scm>
+		<url>https://github.com/apache/flink-connector-mongodb</url>
+		<connection>git@github.com:apache/flink-connector-mongodb.git</connection>
+		<developerConnection>scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git</developerConnection>
+	</scm>
+
+	<properties>
+		<mongodb.version>4.7.1</mongodb.version>
+
+		<flink.version>1.16.0</flink.version>
+		<flink.shaded.version>15.0</flink.shaded.version>
+
+		<junit4.version>4.13.2</junit4.version>
+		<junit5.version>5.8.1</junit5.version>
+		<assertj.version>3.21.0</assertj.version>
+		<archunit.version>0.22.0</archunit.version>
+		<testcontainers.version>1.17.2</testcontainers.version>
+		<mockito.version>3.4.6</mockito.version>
+
+		<japicmp.skip>false</japicmp.skip>
+		<japicmp.referenceVersion>1.15.0</japicmp.referenceVersion>
+
+		<slf4j.version>1.7.36</slf4j.version>
+		<log4j.version>2.17.2</log4j.version>
+
+		<flink.parent.artifactId>flink-connector-mongodb-parent</flink.parent.artifactId>
+	</properties>
+
+	<modules>
+		<module>flink-connector-mongodb</module>
+		<module>flink-sql-connector-mongodb</module>
+		<module>flink-connector-mongodb-e2e-tests</module>
+	</modules>
+
+	<dependencies>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-shaded-force-shading</artifactId>
+			<version>${flink.shaded.version}</version>
+		</dependency>
+
+		<!-- Root dependencies for all projects -->
+
+		<!-- Logging API -->
+		<dependency>
+			<groupId>org.slf4j</groupId>
+			<artifactId>slf4j-api</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- 'javax.annotation' classes like '@Nullable' -->
+		<dependency>
+			<groupId>com.google.code.findbugs</groupId>
+			<artifactId>jsr305</artifactId>
+			<scope>provided</scope>
+		</dependency>
+
+		<!-- Test dependencies -->
+		<dependency>
+			<groupId>org.junit.jupiter</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.junit.vintage</groupId>
+			<artifactId>junit-vintage-engine</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.assertj</groupId>
+			<artifactId>assertj-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-inline</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.mockito</groupId>
+			<artifactId>mockito-core</artifactId>
+			<type>jar</type>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.testcontainers</groupId>
+			<artifactId>junit-jupiter</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- Tests will have log4j as the default logging framework available -->
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-slf4j-impl</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-core</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<!-- API bridge between log4j 1 and 2 -->
+			<groupId>org.apache.logging.log4j</groupId>
+			<artifactId>log4j-1.2-api</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-test-utils-junit</artifactId>
+			<scope>test</scope>
+		</dependency>
+
+		<!-- ArchUit test dependencies -->
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-test</artifactId>
+			<scope>test</scope>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.flink</groupId>
+			<artifactId>flink-architecture-tests-production</artifactId>
+			<scope>test</scope>
+		</dependency>
+	</dependencies>
+
+	<dependencyManagement>
+		<dependencies>
+			<dependency>
+				<groupId>org.mongodb</groupId>
+				<artifactId>mongodb-driver-sync</artifactId>
+				<version>${mongodb.version}</version>
+			</dependency>
+
+			<!-- Flink dependencies -->
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-test-utils</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-test-utils-junit</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<!-- Flink ArchUnit -->
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-base</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-test</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.flink</groupId>
+				<artifactId>flink-architecture-tests-production</artifactId>
+				<version>${flink.version}</version>
+				<scope>test</scope>
+			</dependency>
+
+			<!-- This manages the 'javax.annotation' annotations (JSR305) -->
+			<dependency>
+				<groupId>com.google.code.findbugs</groupId>
+				<artifactId>jsr305</artifactId>
+				<version>1.3.9</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.slf4j</groupId>
+				<artifactId>slf4j-api</artifactId>
+				<version>${slf4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-slf4j-impl</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-api</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-core</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>
+
+			<dependency>
+				<!-- API bridge between log4j 1 and 2 -->
+				<groupId>org.apache.logging.log4j</groupId>
+				<artifactId>log4j-1.2-api</artifactId>
+				<version>${log4j.version}</version>
+			</dependency>

Review Comment:
   Removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1012085964


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/MongoSourceBuilder.java:
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.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.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy;
+import org.apache.flink.connector.mongodb.source.reader.deserializer.MongoDeserializationSchema;
+
+import org.bson.BsonDocument;
+
+import java.util.Arrays;
+import java.util.List;
+
+import static org.apache.flink.util.CollectionUtil.isNullOrEmpty;
+import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * The builder class for {@link MongoSource} to make it easier for the users to construct a {@link
+ * MongoSource}.
+ *
+ * @param <OUT> The output type of the source.
+ */
+@PublicEvolving
+public class MongoSourceBuilder<OUT> {
+
+    private final MongoConnectionOptions.MongoConnectionOptionsBuilder connectionOptionsBuilder;
+    private final MongoReadOptions.MongoReadOptionsBuilder readOptionsBuilder;
+
+    private List<String> projectedFields;
+    private int limit = -1;
+    private MongoDeserializationSchema<OUT> deserializationSchema;
+
+    MongoSourceBuilder() {
+        this.connectionOptionsBuilder = MongoConnectionOptions.builder();
+        this.readOptionsBuilder = MongoReadOptions.builder();
+    }
+
+    /**
+     * Sets the connection string of MongoDB.
+     *
+     * @param uri connection string of MongoDB
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setUri(String uri) {
+        connectionOptionsBuilder.setUri(uri);
+        return this;
+    }
+
+    /**
+     * Sets the database to sink of MongoDB.
+     *
+     * @param database the database to sink of MongoDB.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setDatabase(String database) {
+        connectionOptionsBuilder.setDatabase(database);
+        return this;
+    }
+
+    /**
+     * Sets the collection to sink of MongoDB.
+     *
+     * @param collection the collection to sink of MongoDB.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setCollection(String collection) {
+        connectionOptionsBuilder.setCollection(collection);
+        return this;
+    }
+
+    /**
+     * Sets the number of documents should be fetched per round-trip when reading.
+     *
+     * @param fetchSize the number of documents should be fetched per round-trip when reading.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setFetchSize(int fetchSize) {
+        readOptionsBuilder.setFetchSize(fetchSize);
+        return this;
+    }
+
+    /**
+     * Sets the batch size of MongoDB find cursor.
+     *
+     * @param cursorBatchSize the max batch size of find cursor.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setCursorBatchSize(int cursorBatchSize) {
+        readOptionsBuilder.setCursorBatchSize(cursorBatchSize);
+        return this;
+    }
+
+    /**
+     * Set this option to true to prevent cursor timeout (defaults to 10 minutes).
+     *
+     * @param noCursorTimeout Set this option to true to prevent cursor timeout.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setNoCursorTimeout(boolean noCursorTimeout) {
+        readOptionsBuilder.setNoCursorTimeout(noCursorTimeout);
+        return this;
+    }
+
+    /**
+     * Sets the partition strategy.
+     *
+     * @param partitionStrategy the strategy of a partition.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setPartitionStrategy(PartitionStrategy partitionStrategy) {
+        readOptionsBuilder.setPartitionStrategy(partitionStrategy);
+        return this;
+    }
+
+    /**
+     * Sets the partition size of MongoDB split.
+     *
+     * @param partitionSize the memory size of a partition.
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setPartitionSize(MemorySize partitionSize) {
+        readOptionsBuilder.setPartitionSize(partitionSize);
+        return this;
+    }
+
+    /**
+     * Sets the samples size per partition only effective for sample partition strategy.
+     *
+     * @param samplesPerPartition the samples size per partition
+     * @return this builder
+     */
+    public MongoSourceBuilder<OUT> setSamplesPerPartition(int samplesPerPartition) {
+        readOptionsBuilder.setSamplesPerPartition(samplesPerPartition);
+        return this;
+    }
+
+    /**
+     * Sets the limit of documents to read.

Review Comment:
   If limit is not set, the documents of the entire collection will be read.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1012083456


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/serializer/MongoSerializationSchema.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer.serializer;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+
+import java.io.Serializable;
+
+/**
+ * The serialization schema for how to serialize records into MongoDB.
+ *
+ * @param <IN> The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema<IN> extends Serializable {
+
+    /**
+     * Initialization method for the schema. It is called before the actual working methods {@link
+     * #serialize(Object, MongoSinkContext)} and thus suitable for one-time setup work.
+     *
+     * <p>The provided {@link SerializationSchema.InitializationContext} can be used to access
+     * additional features such as registering user metrics.
+     *
+     * @param initializationContext Contextual information that can be used during initialization.
+     * @param sinkContext Runtime information i.e. partitions, subtaskId.
+     * @param sinkConfiguration All the configure options for the MongoDB sink. You can add custom

Review Comment:
   Sorry, I missed this comment.
   We can set these options by `MongoSinkBuilder`.
   Here's an example:
   ```java
   MongoSink<String> sink =
                   MongoSink.<String>builder()
                           .setUri("mongodb://127.0.0.1:27017")
                           .setDatabase("test")
                           .setCollection("test")
                           .setBulkFlushMaxActions(5)
                           .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
                           .setSerializationSchema(new MongoSerializationSchema<String>() {
                               @Override
                               public void open(
                                       SerializationSchema.InitializationContext initializationContext,
                                       MongoSinkContext sinkContext,
                                       MongoWriteOptions sinkConfiguration) throws Exception {
                                   MongoSerializationSchema.super.open(
                                           initializationContext,
                                           sinkContext,
                                           sinkConfiguration);
                                   // We can access contexts and sinkConfiguration 
                                   // set through the builder here.
                               }
   
                               @Override
                               public WriteModel<BsonDocument> serialize(
                                       String element,
                                       MongoSinkContext sinkContext) {
                                   return new InsertOneModel<>(BsonDocument.parse(element));
                               }
                           })
                           .build();
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017698998


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                retryIntervalMs,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private Integer parallelism;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkArgument(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkArgument(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0,
+                    "The sink max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the retry interval if writing records to database failed.
+         *
+         * @param retryIntervalMs the retry time interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setRetryInterval(long retryIntervalMs) {
+            checkArgument(
+                    retryIntervalMs > 0,
+                    "The retry interval (in milliseconds) must be larger than 0.");
+            this.retryIntervalMs = retryIntervalMs;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#AT_LEAST_ONCE}
+         *
+         * @param deliveryGuarantee which describes the record emission behaviour
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+            checkArgument(
+                    deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+                    "Mongo sink does not support the EXACTLY_ONCE guarantee.");
+            this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+            return this;
+        }
+
+        /**
+         * Sets the parallelism of the Mongo sink operator. By default, the parallelism is
+         * determined by the framework using the same parallelism of the upstream chained operator.
+         */
+        public MongoWriteOptionsBuilder setParallelism(int parallelism) {

Review Comment:
   I'll ask around why this parallelism setting exists. :+1: 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017041964


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/converter/BsonToRowDataConverters.java:
##########
@@ -0,0 +1,681 @@
+/*
+ * 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.converter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.GenericArrayData;
+import org.apache.flink.table.data.GenericMapData;
+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.data.TimestampData;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.logical.utils.LogicalTypeUtils;
+
+import com.mongodb.internal.HexUtils;
+import org.bson.BsonBinary;
+import org.bson.BsonBinarySubType;
+import org.bson.BsonDocument;
+import org.bson.BsonRegularExpression;
+import org.bson.BsonUndefined;
+import org.bson.BsonValue;
+import org.bson.codecs.BsonArrayCodec;
+import org.bson.codecs.EncoderContext;
+import org.bson.json.JsonWriter;
+import org.bson.types.Decimal128;
+
+import java.io.Serializable;
+import java.io.StringWriter;
+import java.io.Writer;
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static java.time.format.DateTimeFormatter.ISO_OFFSET_DATE_TIME;
+import static org.apache.flink.util.Preconditions.checkArgument;
+
+/** Tool class used to convert from {@link BsonValue} to {@link RowData}. * */
+@Internal
+public class BsonToRowDataConverters {
+
+    // -------------------------------------------------------------------------------------
+    // Runtime Converters
+    // -------------------------------------------------------------------------------------
+
+    /**
+     * Runtime converter that converts {@link BsonValue} into objects of Flink Table & SQL internal
+     * data structures.
+     */
+    @FunctionalInterface
+    public interface BsonToRowDataConverter extends Serializable {
+        Object convert(BsonValue bsonValue);
+    }
+
+    // --------------------------------------------------------------------------------
+    // IMPORTANT! We use anonymous classes instead of lambdas for a reason here. It is
+    // necessary because the maven shade plugin cannot relocate classes in
+    // SerializedLambdas (MSHADE-260). On the other hand we want to relocate Bson for
+    // sql-connector uber jars.
+    // --------------------------------------------------------------------------------
+
+    /** Creates a runtime converter which is null safe. */
+    public static BsonToRowDataConverter createNullableConverter(LogicalType type) {
+        return wrapIntoNullableInternalConverter(createConverter(type));
+    }
+
+    private static BsonToRowDataConverter wrapIntoNullableInternalConverter(
+            BsonToRowDataConverter bsonToRowDataConverter) {
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (bsonValue == null || bsonValue.isNull() || bsonValue instanceof BsonUndefined) {
+                    return null;
+                }
+                if (bsonValue.isDecimal128() && bsonValue.asDecimal128().getValue().isNaN()) {
+                    return null;
+                }
+                return bsonToRowDataConverter.convert(bsonValue);
+            }
+        };
+    }
+
+    /** Creates a runtime converter which assuming input object is not null. */
+    private static BsonToRowDataConverter createConverter(LogicalType type) {
+        switch (type.getTypeRoot()) {
+            case NULL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return null;
+                    }
+                };
+            case BOOLEAN:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBoolean(bsonValue);
+                    }
+                };
+            case TINYINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToTinyInt(bsonValue);
+                    }
+                };
+            case SMALLINT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToSmallInt(bsonValue);
+                    }
+                };
+            case INTEGER:
+            case INTERVAL_YEAR_MONTH:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToInt(bsonValue);
+                    }
+                };
+            case BIGINT:
+            case INTERVAL_DAY_TIME:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToLong(bsonValue);
+                    }
+                };
+            case TIMESTAMP_WITHOUT_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromLocalDateTime(convertToLocalDateTime(bsonValue));
+                    }
+                };
+            case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return TimestampData.fromInstant(convertToInstant(bsonValue));
+                    }
+                };
+            case FLOAT:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToFloat(bsonValue);
+                    }
+                };
+            case DOUBLE:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToDouble(bsonValue);
+                    }
+                };
+            case CHAR:
+            case VARCHAR:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return StringData.fromString(convertToString(bsonValue));
+                    }
+                };
+            case BINARY:
+            case VARBINARY:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        return convertToBinary(bsonValue);
+                    }
+                };
+            case DECIMAL:
+                return new BsonToRowDataConverter() {
+                    private static final long serialVersionUID = 1L;
+
+                    @Override
+                    public Object convert(BsonValue bsonValue) {
+                        DecimalType decimalType = (DecimalType) type;
+                        BigDecimal decimalValue = convertToBigDecimal(bsonValue);
+                        return DecimalData.fromBigDecimal(
+                                decimalValue, decimalType.getPrecision(), decimalType.getScale());
+                    }
+                };
+            case ROW:
+                return createRowConverter((RowType) type);
+            case ARRAY:
+                return createArrayConverter((ArrayType) type);
+            case MAP:
+                return createMapConverter((MapType) type);
+            case MULTISET:
+            case RAW:
+            default:
+                throw new UnsupportedOperationException("Unsupported type: " + type);
+        }
+    }
+
+    private static BsonToRowDataConverter createRowConverter(RowType rowType) {
+        final BsonToRowDataConverter[] fieldConverters =
+                rowType.getFields().stream()
+                        .map(RowType.RowField::getType)
+                        .map(BsonToRowDataConverters::createNullableConverter)
+                        .toArray(BsonToRowDataConverter[]::new);
+        final int arity = rowType.getFieldCount();
+        final String[] fieldNames = rowType.getFieldNames().toArray(new String[0]);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                GenericRowData row = new GenericRowData(arity);
+                for (int i = 0; i < arity; i++) {
+                    String fieldName = fieldNames[i];
+                    BsonValue fieldValue = document.get(fieldName);
+                    Object convertedField = fieldConverters[i].convert(fieldValue);
+                    row.setField(i, convertedField);
+                }
+                return row;
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createArrayConverter(ArrayType arrayType) {
+        final Class<?> elementClass =
+                LogicalTypeUtils.toInternalConversionClass(arrayType.getElementType());
+        final BsonToRowDataConverter elementConverter =
+                createNullableConverter(arrayType.getElementType());
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isArray()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to arrayType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                List<BsonValue> in = bsonValue.asArray();
+                final Object[] elementArray = (Object[]) Array.newInstance(elementClass, in.size());
+                for (int i = 0; i < in.size(); i++) {
+                    elementArray[i] = elementConverter.convert(in.get(i));
+                }
+                return new GenericArrayData(elementArray);
+            }
+        };
+    }
+
+    private static BsonToRowDataConverter createMapConverter(MapType mapType) {
+        LogicalType keyType = mapType.getKeyType();
+        checkArgument(keyType.supportsInputConversion(String.class));
+
+        LogicalType valueType = mapType.getValueType();
+        BsonToRowDataConverter valueConverter = createNullableConverter(valueType);
+
+        return new BsonToRowDataConverter() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public Object convert(BsonValue bsonValue) {
+                if (!bsonValue.isDocument()) {
+                    throw new IllegalArgumentException(
+                            "Unable to convert to rowType from unexpected value '"
+                                    + bsonValue
+                                    + "' of type "
+                                    + bsonValue.getBsonType());
+                }
+
+                BsonDocument document = bsonValue.asDocument();
+                Map<StringData, Object> map = new HashMap<>();
+                for (String key : document.keySet()) {
+                    map.put(StringData.fromString(key), valueConverter.convert(document.get(key)));
+                }
+                return new GenericMapData(map);
+            }
+        };
+    }
+
+    private static boolean convertToBoolean(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue();
+        }
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue() == 1;
+        }
+        if (bsonValue.isInt64()) {
+            return bsonValue.asInt64().getValue() == 1L;
+        }
+        if (bsonValue.isString()) {
+            return Boolean.parseBoolean(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to boolean from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static byte convertToTinyInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (byte) 1 : (byte) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (byte) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (byte) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Byte.parseByte(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to tinyint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static short convertToSmallInt(BsonValue bsonValue) {
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? (short) 1 : (short) 0;
+        }
+        if (bsonValue.isInt32()) {
+            return (short) bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {
+            return (short) bsonValue.asInt64().getValue();
+        }
+        if (bsonValue.isString()) {
+            return Short.parseShort(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to smallint from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static int convertToInt(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().intValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.intValue();
+            } else if (decimal128Value.isNegative()) {
+                return Integer.MIN_VALUE;
+            } else {
+                return Integer.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isDateTime()) {
+            return (int) Instant.ofEpochMilli(bsonValue.asDateTime().getValue()).getEpochSecond();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime();
+        }
+        if (bsonValue.isString()) {
+            return Integer.parseInt(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to integer from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static long convertToLong(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().longValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.longValue();
+            } else if (decimal128Value.isNegative()) {
+                return Long.MIN_VALUE;
+            } else {
+                return Long.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1L : 0L;
+        }
+        if (bsonValue.isDateTime()) {
+            return bsonValue.asDateTime().getValue();
+        }
+        if (bsonValue.isTimestamp()) {
+            return bsonValue.asTimestamp().getTime() * 1000L;
+        }
+        if (bsonValue.isString()) {
+            return Long.parseLong(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to long from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static double convertToDouble(BsonValue bsonValue) {
+        if (bsonValue.isNumber()) {
+            return bsonValue.asNumber().doubleValue();
+        }
+        if (bsonValue.isDecimal128()) {
+            Decimal128 decimal128Value = bsonValue.asDecimal128().decimal128Value();
+            if (decimal128Value.isFinite()) {
+                return decimal128Value.doubleValue();
+            } else if (decimal128Value.isNegative()) {
+                return -Double.MAX_VALUE;
+            } else {
+                return Double.MAX_VALUE;
+            }
+        }
+        if (bsonValue.isBoolean()) {
+            return bsonValue.asBoolean().getValue() ? 1 : 0;
+        }
+        if (bsonValue.isString()) {
+            return Double.parseDouble(bsonValue.asString().getValue());
+        }
+        throw new IllegalArgumentException(
+                "Unable to convert to double from unexpected value '"
+                        + bsonValue
+                        + "' of type "
+                        + bsonValue.getBsonType());
+    }
+
+    private static float convertToFloat(BsonValue bsonValue) {
+        if (bsonValue.isInt32()) {
+            return bsonValue.asInt32().getValue();
+        }
+        if (bsonValue.isInt64()) {

Review Comment:
   The reason why there are some loose conversions here is that MongoDB doesn't have fixed type constraints when creating tables like MySQL. The field has same name in MongoDB collection can hold any type, so we made some compatible conversions to make it work as well as possible in flink sql.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1017744379


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/config/MongoWriteOptions.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * 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.sink.config;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+
+import javax.annotation.Nullable;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_INTERVAL;
+import static org.apache.flink.connector.mongodb.table.MongoConnectorOptions.BULK_FLUSH_MAX_ACTIONS;
+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.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Configurations for MongoSink to control write operations. All the options list here could be
+ * configured by {@link MongoWriteOptionsBuilder}.
+ */
+@PublicEvolving
+public final class MongoWriteOptions implements Serializable {
+
+    private static final long serialVersionUID = 1L;
+
+    private final int bulkFlushMaxActions;
+    private final long bulkFlushIntervalMs;
+    private final int maxRetryTimes;
+    private final long retryIntervalMs;
+    private final DeliveryGuarantee deliveryGuarantee;
+    private final Integer parallelism;
+
+    private MongoWriteOptions(
+            int bulkFlushMaxActions,
+            long bulkFlushIntervalMs,
+            int maxRetryTimes,
+            long retryIntervalMs,
+            DeliveryGuarantee deliveryGuarantee,
+            @Nullable Integer parallelism) {
+        this.bulkFlushMaxActions = bulkFlushMaxActions;
+        this.bulkFlushIntervalMs = bulkFlushIntervalMs;
+        this.maxRetryTimes = maxRetryTimes;
+        this.retryIntervalMs = retryIntervalMs;
+        this.deliveryGuarantee = deliveryGuarantee;
+        this.parallelism = parallelism;
+    }
+
+    public int getBulkFlushMaxActions() {
+        return bulkFlushMaxActions;
+    }
+
+    public long getBulkFlushIntervalMs() {
+        return bulkFlushIntervalMs;
+    }
+
+    public int getMaxRetryTimes() {
+        return maxRetryTimes;
+    }
+
+    public long getRetryIntervalMs() {
+        return retryIntervalMs;
+    }
+
+    public DeliveryGuarantee getDeliveryGuarantee() {
+        return deliveryGuarantee;
+    }
+
+    @Nullable
+    public Integer getParallelism() {
+        return parallelism;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (o == null || getClass() != o.getClass()) {
+            return false;
+        }
+        MongoWriteOptions that = (MongoWriteOptions) o;
+        return bulkFlushMaxActions == that.bulkFlushMaxActions
+                && bulkFlushIntervalMs == that.bulkFlushIntervalMs
+                && maxRetryTimes == that.maxRetryTimes
+                && retryIntervalMs == that.retryIntervalMs
+                && deliveryGuarantee == that.deliveryGuarantee
+                && Objects.equals(parallelism, that.parallelism);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(
+                bulkFlushMaxActions,
+                bulkFlushIntervalMs,
+                maxRetryTimes,
+                retryIntervalMs,
+                deliveryGuarantee,
+                parallelism);
+    }
+
+    public static MongoWriteOptionsBuilder builder() {
+        return new MongoWriteOptionsBuilder();
+    }
+
+    /** Builder for {@link MongoWriteOptions}. */
+    @PublicEvolving
+    public static class MongoWriteOptionsBuilder {
+        private int bulkFlushMaxActions = BULK_FLUSH_MAX_ACTIONS.defaultValue();
+        private long bulkFlushIntervalMs = BULK_FLUSH_INTERVAL.defaultValue().toMillis();
+        private int maxRetryTimes = SINK_MAX_RETRIES.defaultValue();
+        private long retryIntervalMs = SINK_RETRY_INTERVAL.defaultValue().toMillis();
+        private DeliveryGuarantee deliveryGuarantee = DeliveryGuarantee.AT_LEAST_ONCE;
+        private Integer parallelism;
+
+        private MongoWriteOptionsBuilder() {}
+
+        /**
+         * Sets the maximum number of actions to buffer for each bulk request. You can pass -1 to
+         * disable it. The default flush size is 1000.
+         *
+         * @param numMaxActions the maximum number of actions to buffer per bulk request.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushMaxActions(int numMaxActions) {
+            checkArgument(
+                    numMaxActions == -1 || numMaxActions > 0,
+                    "Max number of buffered actions must be larger than 0.");
+            this.bulkFlushMaxActions = numMaxActions;
+            return this;
+        }
+
+        /**
+         * Sets the bulk flush interval, in milliseconds. You can pass -1 to disable it.
+         *
+         * @param intervalMillis the bulk flush interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setBulkFlushIntervalMs(long intervalMillis) {
+            checkArgument(
+                    intervalMillis == -1 || intervalMillis >= 0,
+                    "Interval (in milliseconds) between each flush must be larger than "
+                            + "or equal to 0.");
+            this.bulkFlushIntervalMs = intervalMillis;
+            return this;
+        }
+
+        /**
+         * Sets the max retry times if writing records failed.
+         *
+         * @param maxRetryTimes the max retry times.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setMaxRetryTimes(int maxRetryTimes) {
+            checkArgument(
+                    maxRetryTimes >= 0,
+                    "The sink max retry times must be larger than or equal to 0.");
+            this.maxRetryTimes = maxRetryTimes;
+            return this;
+        }
+
+        /**
+         * Sets the retry interval if writing records to database failed.
+         *
+         * @param retryIntervalMs the retry time interval, in milliseconds.
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setRetryInterval(long retryIntervalMs) {
+            checkArgument(
+                    retryIntervalMs > 0,
+                    "The retry interval (in milliseconds) must be larger than 0.");
+            this.retryIntervalMs = retryIntervalMs;
+            return this;
+        }
+
+        /**
+         * Sets the wanted {@link DeliveryGuarantee}. The default delivery guarantee is {@link
+         * DeliveryGuarantee#AT_LEAST_ONCE}
+         *
+         * @param deliveryGuarantee which describes the record emission behaviour
+         * @return this builder
+         */
+        public MongoWriteOptionsBuilder setDeliveryGuarantee(DeliveryGuarantee deliveryGuarantee) {
+            checkArgument(
+                    deliveryGuarantee != DeliveryGuarantee.EXACTLY_ONCE,
+                    "Mongo sink does not support the EXACTLY_ONCE guarantee.");
+            this.deliveryGuarantee = checkNotNull(deliveryGuarantee);
+            return this;
+        }
+
+        /**
+         * Sets the parallelism of the Mongo sink operator. By default, the parallelism is
+         * determined by the framework using the same parallelism of the upstream chained operator.
+         */
+        public MongoWriteOptionsBuilder setParallelism(int parallelism) {

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#issuecomment-1344276831

   Thanks @twalthr for the detailed review.
   Please help review it again when you have time.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1052185934


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableFactory.java:
##########
@@ -151,16 +146,11 @@ public DynamicTableSink createDynamicTableSink(Context context) {
         MongoConfiguration config = new MongoConfiguration(helper.getOptions());
         helper.validate();
 
-        ResolvedSchema schema = context.getCatalogTable().getResolvedSchema();
-        SerializableFunction<RowData, BsonValue> keyExtractor =
-                MongoKeyExtractor.createKeyExtractor(schema);
-
         return new MongoDynamicTableSink(
                 getConnectionOptions(config),
                 getWriteOptions(config),
                 config.getSinkParallelism(),
-                context.getPhysicalRowDataType(),
-                keyExtractor);
+                context.getCatalogTable().getResolvedSchema());

Review Comment:
   OK



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] twalthr commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
twalthr commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1051933589


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/converter/MongoConvertersTest.java:
##########
@@ -146,28 +146,31 @@ public void testConvertBsonToRowData() {
                 GenericRowData.of(
                         StringData.fromString(oid.toHexString()),
                         StringData.fromString("string"),
-                        StringData.fromString(uuid.toString()),
+                        StringData.fromString(
+                                "{\"_value\": {\"$binary\": {\"base64\": \"gR+qXamERr+L0IyxvO9daQ==\", \"subType\": \"04\"}}}"),
                         2,
                         3L,
                         4.1d,
                         DecimalData.fromBigDecimal(new BigDecimal("5.1"), 10, 2),
                         false,
                         TimestampData.fromEpochMillis(now.getEpochSecond() * 1000),
+                        TimestampData.fromEpochMillis(now.toEpochMilli()),
                         StringData.fromString(
-                                OffsetDateTime.ofInstant(
-                                                Instant.ofEpochMilli(now.toEpochMilli()),
-                                                ZoneOffset.UTC)
-                                        .format(ISO_OFFSET_DATE_TIME)),
-                        StringData.fromString("/^9$/i"),
-                        StringData.fromString("function() { return 10; }"),
-                        StringData.fromString("function() { return 11; }"),
-                        StringData.fromString("12"),
-                        StringData.fromString(oid.toHexString()),
+                                "{\"_value\": {\"$regularExpression\": {\"pattern\": \"^9$\", \"options\": \"i\"}}}"),

Review Comment:
   shouldn't the sting be just `{\"$regularExpression\": {\"pattern\": \"^9$\", \"options\": \"i\"}}` without the outer `_value`? or is this important for the round trip?



##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSink.java:
##########
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.table;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.MongoSink;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters;
+import org.apache.flink.connector.mongodb.table.converter.RowDataToBsonConverters.RowDataToBsonConverter;
+import org.apache.flink.connector.mongodb.table.serialization.MongoRowDataSerializationSchema;
+import org.apache.flink.table.connector.ChangelogMode;
+import org.apache.flink.table.connector.sink.DynamicTableSink;
+import org.apache.flink.table.connector.sink.SinkV2Provider;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.function.SerializableFunction;
+
+import org.bson.BsonValue;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** A {@link DynamicTableSink} for MongoDB. */
+@Internal
+public class MongoDynamicTableSink implements DynamicTableSink {
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    @Nullable private final Integer parallelism;
+    private final DataType physicalRowDataType;
+    private final SerializableFunction<RowData, BsonValue> keyExtractor;
+
+    public MongoDynamicTableSink(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            @Nullable Integer parallelism,
+            DataType physicalRowDataType,
+            SerializableFunction<RowData, BsonValue> keyExtractor) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.parallelism = parallelism;
+        this.physicalRowDataType = checkNotNull(physicalRowDataType);
+        this.keyExtractor = checkNotNull(keyExtractor);
+    }
+
+    @Override
+    public ChangelogMode getChangelogMode(ChangelogMode requestedMode) {
+        return ChangelogMode.upsert();

Review Comment:
   What I had in mind was something like:
   ```
   if (hasPrimaryKey(schema)) {
     return ChangelogMode.upsert();
   } else {
     return ChangelogMode.insertOnly();
   }
   ```
   
   This would ensure that the planner will not allow updates to be written into a sink without primary key defined.



##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSourceITCase.java:
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.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 org.bson.BsonArray;
+import org.bson.BsonBinary;
+import org.bson.BsonBoolean;
+import org.bson.BsonDateTime;
+import org.bson.BsonDecimal128;
+import org.bson.BsonDocument;
+import org.bson.BsonDouble;
+import org.bson.BsonInt32;
+import org.bson.BsonInt64;
+import org.bson.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.BeforeEach;
+import org.junit.jupiter.api.Test;
+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.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+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.table.factories.FactoryUtil.CONNECTOR;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase for {@link MongoDynamicTableSource}. */
+@Testcontainers
+public class MongoDynamicTableSourceITCase {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MongoDynamicTableSinkITCase.class);
+
+    @RegisterExtension
+    static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(1)
+                            .build());
+
+    @Container
+    private static final MongoDBContainer MONGO_CONTAINER =
+            MongoTestUtil.createMongoDBContainer(LOG);
+
+    public static final String TEST_DATABASE = "test";
+    public static final String TEST_COLLECTION = "mongo_table_source";
+
+    private static MongoClient mongoClient;
+
+    public static StreamExecutionEnvironment env;
+    public static StreamTableEnvironment tEnv;
+
+    @BeforeAll
+    static void beforeAll() {
+        mongoClient = MongoClients.create(MONGO_CONTAINER.getConnectionString());
+
+        MongoCollection<BsonDocument> coll =
+                mongoClient
+                        .getDatabase(TEST_DATABASE)
+                        .getCollection(TEST_COLLECTION)
+                        .withDocumentClass(BsonDocument.class);
+
+        List<BsonDocument> testRecords = Arrays.asList(createTestData(1), createTestData(2));
+        coll.insertMany(testRecords);
+    }
+
+    @AfterAll
+    static void afterAll() {
+        if (mongoClient != null) {
+            mongoClient.close();
+        }
+    }
+
+    @BeforeEach
+    void before() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        tEnv = StreamTableEnvironment.create(env);
+    }
+
+    @Test
+    public void testSource() {
+        tEnv.executeSql(createTestDDl(null));
+
+        Iterator<Row> collected = tEnv.executeSql("SELECT * FROM mongo_source").collect();
+        List<String> result =
+                CollectionUtil.iteratorToList(collected).stream()
+                        .map(Row::toString)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        List<String> expected =
+                Stream.of(
+                                "+I[1, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]",
+                                "+I[2, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]")
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        assertThat(result).isEqualTo(expected);
+    }
+
+    @Test
+    public void testProject() {
+        tEnv.executeSql(createTestDDl(null));
+
+        Iterator<Row> collected = tEnv.executeSql("SELECT f1, f13 FROM mongo_source").collect();
+        List<String> result =
+                CollectionUtil.iteratorToList(collected).stream()
+                        .map(Row::toString)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        List<String> expected =
+                Stream.of("+I[2, +I[13]]", "+I[2, +I[13]]").sorted().collect(Collectors.toList());
+
+        assertThat(result).isEqualTo(expected);
+    }
+
+    @Test
+    public void testLimit() {
+        tEnv.executeSql(createTestDDl(null));
+
+        Iterator<Row> collected = tEnv.executeSql("SELECT * FROM mongo_source LIMIT 1").collect();
+        List<String> result =
+                CollectionUtil.iteratorToList(collected).stream()
+                        .map(Row::toString)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+        Set<String> expected = new HashSet<>();
+        expected.add(
+                "+I[1, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]");
+        expected.add(
+                "+I[2, 2, false, [3], 4, 5, 6, 2022-09-07T10:25:28.127Z, 2022-09-07T10:25:28Z, 0.9, 1.0, 1.10, {k=12}, +I[13], [14_1, 14_2], [+I[15_1], +I[15_2]]]");
+
+        assertThat(result).hasSize(1);
+        assertThat(result).containsAnyElementsOf(expected);
+    }
+
+    @ParameterizedTest
+    @EnumSource(Caching.class)
+    public void testLookupJoin(Caching caching) throws Exception {
+        // Create MongoDB lookup table
+        Map<String, String> 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<Row> 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<Row> 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<String> result =
+                    CollectionUtil.iteratorToList(iterator).stream()
+                            .map(Row::toString)
+                            .sorted()
+                            .collect(Collectors.toList());
+            List<String> 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<String, LookupCacheManager.RefCountedCache> 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<RowData, Collection<RowData>> 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<String, String> extraOptions) {
+        Map<String, String> options = new HashMap<>();
+        options.put(CONNECTOR.key(), "mongodb");
+        options.put(URI.key(), MONGO_CONTAINER.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,",

Review Comment:
   Thanks for the detailed explanation. I thought `_id` always contains an `ObjectId` that might be generated by the user -provided `_id`. Your explanation makes sense to me. So only the the missing data types should limit a round-trip but this we can tackle in future versions.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] zentol merged pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by "zentol (via GitHub)" <gi...@apache.org>.
zentol merged PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [flink-connector-mongodb] Jiabao-Sun commented on a diff in pull request #1: [FLINK-6573][connectors/mongodb] Flink MongoDB Connector

Posted by GitBox <gi...@apache.org>.
Jiabao-Sun commented on code in PR #1:
URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1055087635


##########
flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/converter/MongoConvertersTest.java:
##########
@@ -146,28 +146,31 @@ public void testConvertBsonToRowData() {
                 GenericRowData.of(
                         StringData.fromString(oid.toHexString()),
                         StringData.fromString("string"),
-                        StringData.fromString(uuid.toString()),
+                        StringData.fromString(
+                                "{\"_value\": {\"$binary\": {\"base64\": \"gR+qXamERr+L0IyxvO9daQ==\", \"subType\": \"04\"}}}"),
                         2,
                         3L,
                         4.1d,
                         DecimalData.fromBigDecimal(new BigDecimal("5.1"), 10, 2),
                         false,
                         TimestampData.fromEpochMillis(now.getEpochSecond() * 1000),
+                        TimestampData.fromEpochMillis(now.toEpochMilli()),
                         StringData.fromString(
-                                OffsetDateTime.ofInstant(
-                                                Instant.ofEpochMilli(now.toEpochMilli()),
-                                                ZoneOffset.UTC)
-                                        .format(ISO_OFFSET_DATE_TIME)),
-                        StringData.fromString("/^9$/i"),
-                        StringData.fromString("function() { return 10; }"),
-                        StringData.fromString("function() { return 11; }"),
-                        StringData.fromString("12"),
-                        StringData.fromString(oid.toHexString()),
+                                "{\"_value\": {\"$regularExpression\": {\"pattern\": \"^9$\", \"options\": \"i\"}}}"),

Review Comment:
   Hi @twalthr.
   In a recent commit, read a RegExp as string and write it out as a RegExp to MongoDB again is supported.
   Please help to review it when you have time.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org