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

2023-02-07 Thread via GitHub


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
+ *
+ * Samples the collection to generate partitions.
+ *
+ * Uses the average document size to split the collection into average 
sized chunks
+ *
+ * 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.
+ *
+ * 
+ *   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.
+ *   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)}.
+ * 
+ */
+@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-unsubscr...@flink.apache.org

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



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

2023-02-06 Thread via GitHub


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
+ *
+ * Samples the collection to generate partitions.
+ *
+ * Uses the average document size to split the collection into average 
sized chunks
+ *
+ * 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.
+ *
+ * 
+ *   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.
+ *   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)}.
+ * 
+ */
+@Internal
+public class MongoSampleSplitter {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(MongoSampleSplitter.class);
+
+public static final MongoSampleSplitter INSTANCE = new 
MongoSampleSplitter();
+
+private MongoSampleSplitter() {}
+
+public Collection 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 samples =
+splitContext
+.getMongoCollection()
+.aggregate(
+Arrays.asList(
+Aggregates.sample(numberOfSamples),
+
Aggregates.project(Projections.include(ID_FIELD)),
+
Aggregates.sort(Sorts.ascending(ID_FIELD

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

2023-02-06 Thread via GitHub


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
+ *
+ * Samples the collection to generate partitions.
+ *
+ * Uses the average document size to split the collection into average 
sized chunks
+ *
+ * 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.
+ *
+ * 
+ *   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.
+ *   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)}.
+ * 
+ */
+@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-unsubscr...@flink.apache.org

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



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

2023-02-06 Thread via GitHub


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
+ *
+ * Samples the collection to generate partitions.
+ *
+ * Uses the average document size to split the collection into average 
sized chunks
+ *
+ * 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.
+ *
+ * 
+ *   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.
+ *   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)}.
+ * 
+ */
+@Internal
+public class MongoSampleSplitter {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(MongoSampleSplitter.class);
+
+public static final MongoSampleSplitter INSTANCE = new 
MongoSampleSplitter();
+
+private MongoSampleSplitter() {}
+
+public Collection 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 samples =
+splitContext
+.getMongoCollection()
+.aggregate(
+Arrays.asList(
+Aggregates.sample(numberOfSamples),
+
Aggregates.project(Projections.include(ID_FIELD)),
+
Aggregates.sort(Sorts.ascending(ID_FIELD

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

2023-02-06 Thread via GitHub


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();
+}
+
+

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

2023-02-06 Thread via GitHub


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();
+}
+
+

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

2023-02-06 Thread via GitHub


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 remainingCollections;
+private final List alreadyProcessedCollections;
+private final List remainingScanSplits;
+private final Map 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-unsubscr...@flink.apache.org

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



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

2023-02-06 Thread via GitHub


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

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



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

2023-02-06 Thread via GitHub


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 URI =
+ConfigOptions.key("uri")
+.stringType()
+.noDefaultValue()
+.withDescription("Specifies the connection uri of 
MongoDB.");
+
+public static final ConfigOption DATABASE =
+ConfigOptions.key("database")
+.stringType()
+.noDefaultValue()
+.withDescription("Specifies the database to read or write 
of MongoDB.");
+
+public static final ConfigOption COLLECTION =
+ConfigOptions.key("collection")
+.stringType()
+.noDefaultValue()
+.withDescription("Specifies the collection to read or 
write of MongoDB.");
+
+public static final ConfigOption 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 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-unsubscr...@flink.apache.org

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



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

2023-02-05 Thread via GitHub


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();
+}
+
+

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

2023-02-05 Thread via GitHub


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

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



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

2023-02-05 Thread via GitHub


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

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



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

2023-02-05 Thread via GitHub


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

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



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

2023-02-03 Thread via GitHub


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

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



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

2023-02-03 Thread via GitHub


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
+ *
+ * Samples the collection to generate partitions.
+ *
+ * Uses the average document size to split the collection into average 
sized chunks
+ *
+ * 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.
+ *
+ * 
+ *   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.
+ *   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-unsubscr...@flink.apache.org

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



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

2023-02-03 Thread via GitHub


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 remainingCollections;
+private final List alreadyProcessedCollections;
+private final List remainingScanSplits;
+private final Map 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 getNext() {
+if (!remainingScanSplits.isEmpty()) {
+// return remaining splits firstly
+Iterator 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 splits =
+mongoSplitters.split(new 
MongoNamespace(nextCollection));
+remainingScanSplits.addAll(splits);
+alreadyProcessedCollections.add(nextCollection);
+return getNext();
+} else {
+return Optional.empty();
+}
+}
+}
+
+@Override
+public void addSplitsBack(Collection splits) {
+for (MongoSourceSplit 

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

2023-02-03 Thread via GitHub


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 {

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

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



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

2023-02-03 Thread via GitHub


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 remainingCollections;
+private final List alreadyProcessedCollections;
+private final List remainingScanSplits;
+private final Map 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-unsubscr...@flink.apache.org

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



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

2023-02-03 Thread via GitHub


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

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



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

2023-02-03 Thread via GitHub


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

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



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

2023-02-03 Thread via GitHub


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

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



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

2023-02-03 Thread via GitHub


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

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



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

2022-12-21 Thread GitBox


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

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



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

2022-12-19 Thread GitBox


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

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



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

2022-12-19 Thread GitBox


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

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



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

2022-12-19 Thread GitBox


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

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



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

2022-12-19 Thread GitBox


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

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



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

2022-12-09 Thread GitBox


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()) 

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

2022-12-08 Thread GitBox


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()) 

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

2022-12-08 Thread GitBox


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()) 

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

2022-12-08 Thread GitBox


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 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");
+

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

2022-12-08 Thread GitBox


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 

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

2022-12-08 Thread GitBox


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()) 

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

2022-12-08 Thread GitBox


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 keyExtractor;
+
+public MongoDynamicTableSink(
+MongoConnectionOptions connectionOptions,
+MongoWriteOptions writeOptions,
+@Nullable Integer parallelism,
+DataType physicalRowDataType,
+SerializableFunction 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-unsubscr...@flink.apache.org

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



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

2022-12-08 Thread GitBox


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 

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

2022-12-02 Thread GitBox


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

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



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

2022-11-29 Thread GitBox


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

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



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

2022-11-29 Thread GitBox


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

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



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

2022-11-27 Thread GitBox


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
+ *
+ * Samples the collection to generate partitions.
+ *
+ * Uses the average document size to split the collection into average 
sized chunks
+ *
+ * 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.
+ *
+ * 
+ *   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.
+ *   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 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-unsubscr...@flink.apache.org

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



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

2022-11-27 Thread GitBox


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 fieldNames;
+private final List keyNames;
+
+private final BsonToRowDataConverters.BsonToRowDataConverter 
mongoRowConverter;
+private final RowDataToBsonConverters.RowDataToBsonConverter 
lookupKeyRowConverter;
+
+private transient MongoClient mongoClient;
+
+public MongoRowDataLookupFunction(
+MongoConnectionOptions connectionOptions,
+int maxRetries,
+long retryIntervalMs,
+List fieldNames,
+List fieldTypes,
+List 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-unsubscr...@flink.apache.org

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



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

2022-11-27 Thread GitBox


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 keyExtractor;
+
+public MongoDynamicTableSink(
+MongoConnectionOptions connectionOptions,
+MongoWriteOptions writeOptions,
+@Nullable Integer parallelism,
+DataType physicalRowDataType,
+SerializableFunction 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-unsubscr...@flink.apache.org

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



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

2022-11-27 Thread GitBox


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 keyExtractor;
+
+public MongoDynamicTableSink(
+MongoConnectionOptions connectionOptions,
+MongoWriteOptions writeOptions,
+@Nullable Integer parallelism,
+DataType physicalRowDataType,
+SerializableFunction 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-unsubscr...@flink.apache.org

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



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

2022-11-27 Thread GitBox


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

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



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

2022-11-27 Thread GitBox


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 {
+
+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 projectedFields;
+private final int limit;
+
+private boolean closed = false;
+private boolean finished = false;
+private MongoClient mongoClient;
+private MongoCursor currentCursor;
+private MongoScanSourceSplit currentSplit;
+
+public MongoScanSourceSplitReader(
+MongoConnectionOptions connectionOptions,
+MongoReadOptions readOptions,
+@Nullable List projectedFields,
+int limit,
+SourceReaderContext context) {
+this.connectionOptions = connectionOptions;
+this.readOptions = readOptions;
+this.projectedFields = projectedFields;
+this.limit = limit;
+this.readerContext = context;
+}
+
+@Override
+public RecordsWithSplitIds fetch() throws IOException {
+if (closed) {
+throw new IllegalStateException("Cannot fetch records from a 
closed split reader");
+}
+
+RecordsBySplits.Builder builder = new 
RecordsBySplits.Builder<>();
+
+// Return when no split registered to this reader.
+if (currentSplit == null) {
+return builder.build();
+}
+
+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 

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

2022-11-27 Thread GitBox


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 {
+
+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 projectedFields;
+private final int limit;
+
+private boolean closed = false;
+private boolean finished = false;
+private MongoClient mongoClient;
+private MongoCursor currentCursor;
+private MongoScanSourceSplit currentSplit;
+
+public MongoScanSourceSplitReader(
+MongoConnectionOptions connectionOptions,
+MongoReadOptions readOptions,
+@Nullable List projectedFields,
+int limit,
+SourceReaderContext context) {
+this.connectionOptions = connectionOptions;
+this.readOptions = readOptions;
+this.projectedFields = projectedFields;
+this.limit = limit;
+this.readerContext = context;
+}
+
+@Override
+public RecordsWithSplitIds fetch() throws IOException {
+if (closed) {
+throw new IllegalStateException("Cannot fetch records from a 
closed split reader");
+}
+
+RecordsBySplits.Builder builder = new 
RecordsBySplits.Builder<>();
+
+// Return when no split registered to this reader.
+if (currentSplit == null) {
+return builder.build();
+}
+
+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 

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

2022-11-27 Thread GitBox


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

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



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

2022-11-27 Thread GitBox


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
+ *
+ * Uses the chunks collection and partitions the collection based on the 
sharded collections
+ * chunk ranges.
+ *
+ * The following config collections' read privilege is required.
+ *
+ * 
+ *   config.collections
+ *   config.chunks
+ * 
+ */
+@Internal
+public class MongoShardedSplitter {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+public static final MongoShardedSplitter INSTANCE = new 
MongoShardedSplitter();
+
+private MongoShardedSplitter() {}
+
+public Collection split(MongoSplitContext 
splitContext) {
+MongoNamespace namespace = splitContext.getMongoNamespace();
+MongoClient mongoClient = splitContext.getMongoClient();
+
+List chunks;
+Optional 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 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`?
  

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

2022-11-27 Thread GitBox


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
+ *
+ * Uses the chunks collection and partitions the collection based on the 
sharded collections
+ * chunk ranges.
+ *
+ * The following config collections' read privilege is required.
+ *
+ * 
+ *   config.collections
+ *   config.chunks
+ * 
+ */
+@Internal
+public class MongoShardedSplitter {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+public static final MongoShardedSplitter INSTANCE = new 
MongoShardedSplitter();
+
+private MongoShardedSplitter() {}
+
+public Collection split(MongoSplitContext 
splitContext) {
+MongoNamespace namespace = splitContext.getMongoNamespace();
+MongoClient mongoClient = splitContext.getMongoClient();
+
+List chunks;
+Optional 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 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 

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

2022-11-27 Thread GitBox


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
+ *
+ * Uses the chunks collection and partitions the collection based on the 
sharded collections
+ * chunk ranges.
+ *
+ * The following config collections' read privilege is required.
+ *
+ * 
+ *   config.collections
+ *   config.chunks
+ * 
+ */
+@Internal
+public class MongoShardedSplitter {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(MongoShardedSplitter.class);
+
+public static final MongoShardedSplitter INSTANCE = new 
MongoShardedSplitter();
+
+private MongoShardedSplitter() {}
+
+public Collection split(MongoSplitContext 
splitContext) {
+MongoNamespace namespace = splitContext.getMongoNamespace();
+MongoClient mongoClient = splitContext.getMongoClient();
+
+List chunks;
+Optional 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" : 

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

2022-11-27 Thread GitBox


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 {
+
+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 projectedFields;
+private final int limit;
+
+private boolean closed = false;
+private boolean finished = false;
+private MongoClient mongoClient;
+private MongoCursor currentCursor;
+private MongoScanSourceSplit currentSplit;
+
+public MongoScanSourceSplitReader(
+MongoConnectionOptions connectionOptions,
+MongoReadOptions readOptions,
+@Nullable List projectedFields,
+int limit,
+SourceReaderContext context) {
+this.connectionOptions = connectionOptions;
+this.readOptions = readOptions;
+this.projectedFields = projectedFields;
+this.limit = limit;
+this.readerContext = context;
+}
+
+@Override
+public RecordsWithSplitIds fetch() throws IOException {
+if (closed) {
+throw new IllegalStateException("Cannot fetch records from a 
closed split reader");
+}
+
+RecordsBySplits.Builder builder = new 
RecordsBySplits.Builder<>();
+
+// Return when no split registered to this reader.
+if (currentSplit == null) {
+return builder.build();
+}
+
+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 

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

2022-11-25 Thread GitBox


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

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



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

2022-11-25 Thread GitBox


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  Mongo source split.
+ */
+@Internal
+public interface MongoSourceSplitReader

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

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



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

2022-11-25 Thread GitBox


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 keyExtractor;
+
+public MongoDynamicTableSink(
+MongoConnectionOptions connectionOptions,
+MongoWriteOptions writeOptions,
+@Nullable Integer parallelism,
+DataType physicalRowDataType,
+SerializableFunction 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-unsubscr...@flink.apache.org

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



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

2022-11-25 Thread GitBox


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 URI =
+ConfigOptions.key("uri")
+.stringType()
+.noDefaultValue()
+.withDescription("Specifies the connection uri of 
MongoDB.");
+
+public static final ConfigOption DATABASE =
+ConfigOptions.key("database")
+.stringType()
+.noDefaultValue()
+.withDescription("Specifies the database to read or write 
of MongoDB.");
+
+public static final ConfigOption COLLECTION =
+ConfigOptions.key("collection")
+.stringType()
+.noDefaultValue()
+.withDescription("Specifies the collection to read or 
write of MongoDB.");
+
+public static final ConfigOption 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 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-unsubscr...@flink.apache.org

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



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

2022-11-25 Thread GitBox


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> requiredOptions() {
+final Set> requiredOptions = new HashSet<>();
+requiredOptions.add(URI);
+requiredOptions.add(DATABASE);
+requiredOptions.add(COLLECTION);
+return requiredOptions;
+}
+
+@Override
+public Set> optionalOptions() {
+final Set> 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);
+

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

2022-11-24 Thread GitBox


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 {

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

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



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

2022-11-24 Thread GitBox


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 {

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

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



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

2022-11-24 Thread GitBox


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 {
+
+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 createKeyExtractor(
+ResolvedSchema resolvedSchema) {
+
+Optional primaryKey = resolvedSchema.getPrimaryKey();
+int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+Optional 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-unsubscr...@flink.apache.org

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



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

2022-11-23 Thread GitBox


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 @@
+
+
+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;>
+
+   4.0.0
+
+   
+   flink-connector-mongodb-parent
+   org.apache.flink
+   1.0-SNAPSHOT
+   
+
+   flink-connector-mongodb-e2e-tests
+   Flink : E2E Tests : MongoDB
+
+   
+   
+   org.apache.flink
+   flink-connector-test-utils
+   ${flink.version}
+   test
+   
+
+   
+   
+   org.apache.flink
+   flink-sql-connector-mongodb
+   ${project.version}
+   test
+   
+
+   
+   org.mongodb
+   mongodb-driver-sync
+   test
+   
+
+   
+   org.testcontainers
+   mongodb
+   test
+   
+   
+
+   
+   
+   run-end-to-end-tests
+   
+   
+   
+   
org.apache.maven.plugins
+   
maven-surefire-plugin
+   
+   
+   
end-to-end-tests
+   
integration-test
+   
+   
test
+   
+   
+   

+   
**/*.*
+   

+   
1

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

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



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

2022-11-21 Thread GitBox


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 

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

2022-11-21 Thread GitBox


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

For queries about this service, 

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

2022-11-21 Thread GitBox


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
+ *
+ * Uses the SplitVector command to generate chunks for a collection. eg. 

+ * db.runCommand({splitVector:"inventory.products", keyPattern:{_id:1}, 
maxChunkSize:64})
+ *
+ * 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 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-unsubscr...@flink.apache.org

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



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

2022-11-21 Thread GitBox


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 

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

2022-11-17 Thread GitBox


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 {
+
+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 createKeyExtractor(
+ResolvedSchema resolvedSchema) {
+
+Optional primaryKey = resolvedSchema.getPrimaryKey();
+int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes();
+Optional 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() {
+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())) {
+  

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

2022-11-17 Thread GitBox


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

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



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

2022-11-17 Thread GitBox


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

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



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

2022-11-17 Thread GitBox


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

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



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

2022-11-17 Thread GitBox


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())
+

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

2022-11-17 Thread GitBox


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())
+

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

2022-11-17 Thread GitBox


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())
+

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

2022-11-17 Thread GitBox


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())
+

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

2022-11-17 Thread GitBox


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())
+

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

2022-11-17 Thread GitBox


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

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



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

2022-11-10 Thread GitBox


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 @@
+
+
+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;>
+
+   4.0.0
+
+   
+   flink-connector-mongodb-parent
+   org.apache.flink
+   1.0-SNAPSHOT
+   
+
+   flink-sql-connector-mongodb
+   Flink : Connectors : SQL : MongoDB
+
+   jar
+
+   
+   true
+   
+
+   
+   
+   org.apache.flink
+   flink-connector-mongodb
+   ${project.version}
+   
+   
+
+   
+   
+   
+   org.apache.maven.plugins
+   maven-shade-plugin
+   
+   
+   shade-flink
+   package
+   
+   shade
+   
+   
+   
+   
+   
org.apache.flink:flink-connector-base
+   
org.apache.flink:flink-connector-mongodb
+   
org.mongodb:bson
+   
org.mongodb:mongodb-driver-sync
+   
org.mongodb:mongodb-driver-core
+   
+   
+   
+   
+   
org.mongodb:mongodb-driver-core
+   

+   
META-INF/native-image/**/**.properties

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

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



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

2022-11-10 Thread GitBox


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

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



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

2022-11-09 Thread GitBox


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 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 

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

2022-11-09 Thread GitBox


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  The type of the input elements.
+ */
+@Internal
+public class MongoWriter implements SinkWriter {
+
+private static final Logger LOG = 
LoggerFactory.getLogger(MongoWriter.class);
+
+private final MongoConnectionOptions connectionOptions;
+private final MongoWriteOptions writeOptions;
+private final MongoSerializationSchema serializationSchema;
+private final MongoSinkContext sinkContext;
+private final MailboxExecutor mailboxExecutor;
+private final boolean flushOnCheckpoint;
+private final List> bulkRequests = new 
ArrayList<>();
+private final Collector> 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 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 

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

2022-11-09 Thread GitBox


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
+ *
+ * Uses the SplitVector command to generate chunks for a collection. eg. 

+ * db.runCommand({splitVector:"inventory.products", keyPattern:{_id:1}, 
maxChunkSize:64})
+ *
+ * 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-unsubscr...@flink.apache.org

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



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

2022-11-09 Thread GitBox


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 = 

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

2022-11-09 Thread GitBox


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 = 

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

2022-11-09 Thread GitBox


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

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



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

2022-11-09 Thread GitBox


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 = 

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

2022-11-09 Thread GitBox


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 {
+
+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 remainingCollections = deserializeList(in, 
DataInput::readUTF);
+List alreadyProcessedCollections = deserializeList(in, 
DataInput::readUTF);
+List remainingScanSplits =
+deserializeList(in, i -> 
deserializeMongoScanSourceSplit(version, i));
+
+Map 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 

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

2022-11-08 Thread GitBox


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 remainingCollections;
+
+/**
+ * The paths that are no longer in the enumerator checkpoint, but have 
been processed before.
+ */
+private final List 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-unsubscr...@flink.apache.org

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



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

2022-11-08 Thread GitBox


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 

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

2022-11-08 Thread GitBox


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 

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

2022-11-08 Thread GitBox


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 

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

2022-11-08 Thread GitBox


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

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



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

2022-11-08 Thread GitBox


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();

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

2022-11-08 Thread GitBox


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

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



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

2022-11-08 Thread GitBox


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 @@
+
+
+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;>
+
+   4.0.0
+
+   
+   io.github.zentol.flink
+   flink-connector-parent
+   1.0
+   
+
+   org.apache.flink
+   flink-connector-mongodb-parent
+   1.0-SNAPSHOT
+
+   Flink : Connectors : MongoDB Parent
+   pom
+   https://flink.apache.org
+   2022
+
+   
+   
+   The Apache Software License, Version 2.0
+   
https://www.apache.org/licenses/LICENSE-2.0.txt
+   repo
+   
+   
+
+   
+   https://github.com/apache/flink-connector-mongodb
+   
g...@github.com:apache/flink-connector-mongodb.git
+   
scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git
+   
+
+   
+   4.7.1
+
+   1.16.0
+   15.0
+
+   4.13.2
+   5.8.1
+   3.21.0
+   0.22.0
+   1.17.2
+   3.4.6
+
+   false
+   1.15.0
+
+   1.7.36
+   2.17.2
+
+   
flink-connector-mongodb-parent
+   
+
+   
+   flink-connector-mongodb
+   flink-sql-connector-mongodb
+   flink-connector-mongodb-e2e-tests
+   
+
+   
+   
+   org.apache.flink
+   flink-shaded-force-shading
+   ${flink.shaded.version}
+   
+
+   
+
+   
+   
+   org.slf4j
+   slf4j-api
+   provided
+   
+
+   
+   
+   com.google.code.findbugs
+   jsr305
+   provided
+   
+
+   
+   
+   org.junit.jupiter
+   junit-jupiter
+   test
+   
+
+   
+   org.junit.vintage
+   junit-vintage-engine
+   test
+   
+
+   
+   org.assertj
+   assertj-core
+   test
+   
+
+   
+   org.mockito
+   mockito-inline
+   jar
+   test
+   
+
+   
+   org.mockito
+   mockito-core
+   jar
+   test
+   
+
+   
+   org.testcontainers
+   junit-jupiter
+   test
+   
+
+   
+
+   
+   org.apache.logging.log4j
+   log4j-slf4j-impl
+   test
+   
+
+   
+   org.apache.logging.log4j
+   log4j-api
+   test
+   
+
+   
+   org.apache.logging.log4j
+   log4j-core
+   test
+   
+
+   
+   
+   org.apache.logging.log4j
+   log4j-1.2-api
+   test
+   
+
+   
+   org.apache.flink
+   flink-test-utils-junit
+   test
+   
+
+   
+   
+   org.apache.flink
+   flink-architecture-tests-test
+   test
+   
+   
+   org.apache.flink
+   
flink-architecture-tests-production
+   test
+   
+   
+
+   
+   
+   
+   org.mongodb
+   mongodb-driver-sync
+   ${mongodb.version}
+   
+
+   
+
+   
+   org.apache.flink
+   flink-test-utils
+   ${flink.version}
+   test
+   
+
+   
+   org.apache.flink
+   flink-test-utils-junit
+   ${flink.version}
+   test
+  

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

2022-11-08 Thread GitBox


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  The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema 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.
+ *
+ * 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-unsubscr...@flink.apache.org

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



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

2022-11-08 Thread GitBox


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  The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema 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.
+ *
+ * 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-unsubscr...@flink.apache.org

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



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

2022-11-02 Thread GitBox


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  The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema 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.
+ *
+ * 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 sink =
   MongoSink.builder()
   .setUri("mongodb://127.0.0.1:27017")
   .setDatabase("test")
   .setCollection("test")
   .setBulkFlushMaxActions(5)
   
.setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
   .setSerializationSchema(new 
MongoSerializationSchema() {
   @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 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-unsubscr...@flink.apache.org

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



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

2022-11-02 Thread GitBox


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  The output type of the source.
+ */
+@PublicEvolving
+public class MongoSourceBuilder {
+
+private final MongoConnectionOptions.MongoConnectionOptionsBuilder 
connectionOptionsBuilder;
+private final MongoReadOptions.MongoReadOptionsBuilder readOptionsBuilder;
+
+private List projectedFields;
+private int limit = -1;
+private MongoDeserializationSchema 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 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 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 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 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 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 setNoCursorTimeout(boolean noCursorTimeout) 
{
+readOptionsBuilder.setNoCursorTimeout(noCursorTimeout);
+return this;
+}
+
+/**
+ * Sets the partition strategy.
+ *
+ * @param partitionStrategy the strategy of a partition.
+ * @return this builder
+ */
+public MongoSourceBuilder setPartitionStrategy(PartitionStrategy 
partitionStrategy) {
+readOptionsBuilder.setPartitionStrategy(partitionStrategy);
+return this;
+}
+
+/**
+ * Sets 

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

2022-11-02 Thread GitBox


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  The message type send to MongoDB.
+ */
+@PublicEvolving
+public interface MongoSerializationSchema 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.
+ *
+ * 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 sink =
   MongoSink.builder()
   .setUri("mongodb://127.0.0.1:27017")
   .setDatabase("test")
   .setCollection("test")
   .setBulkFlushMaxActions(5)
   
.setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE)
   .setSerializationSchema(new 
MongoSerializationSchema() {
   @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 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-unsubscr...@flink.apache.org

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



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

2022-11-02 Thread GitBox


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

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



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

2022-11-02 Thread GitBox


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 @@
+
+
+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;>
+
+   4.0.0
+
+   
+   io.github.zentol.flink
+   flink-connector-parent
+   1.0
+   
+
+   org.apache.flink
+   flink-connector-mongodb-parent
+   1.0.0-SNAPSHOT
+
+   Flink : Connectors : MongoDB Parent
+   pom
+   https://flink.apache.org
+   2022
+
+   
+   
+   The Apache Software License, Version 2.0
+   
https://www.apache.org/licenses/LICENSE-2.0.txt
+   repo
+   
+   
+
+   
+   https://github.com/apache/flink-connector-mongodb
+   
g...@github.com:apache/flink-connector-mongodb.git
+   
scm:git:https://gitbox.apache.org/repos/asf/flink-connector-mongodb.git
+   
+
+   
+   4.7.1
+
+   1.16-SNAPSHOT
+   15.0
+
+   4.13.2
+   5.8.1
+   1.3
+   3.21.0
+   0.22.0
+   1.17.2
+   3.4.6
+
+   false
+   1.15.0
+
+   1.7.36
+   2.17.2
+
+   
flink-connector-mongodb-parent
+   
+
+   
+   flink-connector-mongodb
+   flink-sql-connector-mongodb
+   flink-connector-mongodb-e2e-tests
+   
+
+   
+   
+   org.apache.flink
+   flink-shaded-force-shading
+   ${flink.shaded.version}
+   
+
+   
+
+   
+   
+   org.slf4j
+   slf4j-api
+   provided
+   
+
+   
+   
+   com.google.code.findbugs
+   jsr305
+   provided
+   
+
+   
+   
+   org.junit.jupiter
+   junit-jupiter
+   test
+   
+
+   
+   org.junit.vintage
+   junit-vintage-engine
+   test
+   
+
+   
+   org.assertj
+   assertj-core
+   test
+   
+
+   
+   org.hamcrest
+   hamcrest-all
+   test
+   
+
+   
+   org.mockito
+   mockito-inline
+   jar
+   test
+   
+
+   
+   org.mockito
+   mockito-core
+   jar
+   test
+   
+
+   
+   org.testcontainers
+   junit-jupiter
+   test
+   
+
+   
+
+   
+   org.apache.logging.log4j
+   log4j-slf4j-impl
+   test
+   
+
+   
+   org.apache.logging.log4j
+   log4j-api
+   test
+   
+
+   
+   org.apache.logging.log4j
+   log4j-core
+   test
+   
+
+   
+   
+   org.apache.logging.log4j
+   log4j-1.2-api
+   test
+   
+
+   
+   org.apache.flink
+   flink-test-utils-junit
+   test
+   
+
+   
+   
+   org.apache.flink
+   flink-architecture-tests-test
+   test
+   
+   
+   org.apache.flink
+   
flink-architecture-tests-production
+   test
+   
+   
+
+   
+   
+   
+   org.mongodb
+   mongodb-driver-sync

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 

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

2022-11-02 Thread GitBox


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

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



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

2022-11-02 Thread GitBox


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

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



  1   2   >