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-unsubscr...@flink.apache.org

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

Reply via email to