Jiabao-Sun commented on code in PR #1:
URL: 
https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1028370047


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/enumerator/splitter/MongoSplitVectorSplitter.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.source.enumerator.splitter;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.MemorySize;
+import org.apache.flink.connector.mongodb.source.config.MongoReadOptions;
+import org.apache.flink.connector.mongodb.source.split.MongoScanSourceSplit;
+
+import com.mongodb.MongoCommandException;
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import org.apache.commons.collections.CollectionUtils;
+import org.bson.BsonArray;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MAX_KEY;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.BSON_MIN_KEY;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.ERROR_MESSAGE_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.SPLIT_KEYS_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoUtils.UNAUTHORIZED_ERROR;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoUtils.isCommandSucceed;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoUtils.splitVector;
+
+/**
+ * SplitVector Partitioner
+ *
+ * <p>Uses the SplitVector command to generate chunks for a collection. eg. 
<code>
+ * db.runCommand({splitVector:"inventory.products", keyPattern:{_id:1}, 
maxChunkSize:64})</code>
+ *
+ * <p>Requires splitVector privilege.
+ */
+@Internal
+public class MongoSplitVectorSplitter implements MongoSplitters.MongoSplitter {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(MongoSplitVectorSplitter.class);
+
+    public static final MongoSplitVectorSplitter INSTANCE = new 
MongoSplitVectorSplitter();
+
+    private MongoSplitVectorSplitter() {}
+
+    @Override
+    public Collection<MongoScanSourceSplit> split(MongoSplitContext 
splitContext) {
+        MongoClient mongoClient = splitContext.getMongoClient();
+        MongoNamespace namespace = splitContext.getMongoNamespace();
+        MongoReadOptions readOptions = splitContext.getReadOptions();
+
+        MemorySize chunkSize = readOptions.getPartitionSize();
+        // if partition size < 1mb, use 1 mb as chunk size.
+        int maxChunkSizeMB = Math.max(chunkSize.getMebiBytes(), 1);
+
+        BsonDocument keyPattern = new BsonDocument(ID_FIELD, new BsonInt32(1));
+
+        BsonDocument splitResult;
+        try {
+            splitResult = splitVector(mongoClient, namespace, keyPattern, 
maxChunkSizeMB);

Review Comment:
   Thanks @zentol, I checked driver source code of 
`InternalStreamConnection#receiveCommandMessageResponse`. If the execution of 
the command fails, an exception will be thrown, and we don't need to deal with 
other case.
   
   



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

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

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

Reply via email to