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


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/common/utils/MongoUtils.java:
##########
@@ -0,0 +1,161 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.common.utils;
+
+import org.apache.flink.annotation.Internal;
+
+import com.mongodb.MongoNamespace;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoCollection;
+import org.bson.BsonBoolean;
+import org.bson.BsonDocument;
+import org.bson.BsonInt32;
+import org.bson.BsonString;
+import org.bson.conversions.Bson;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+
+import static com.mongodb.client.model.Filters.eq;
+import static com.mongodb.client.model.Filters.or;
+import static com.mongodb.client.model.Projections.excludeId;
+import static com.mongodb.client.model.Projections.fields;
+import static com.mongodb.client.model.Projections.include;
+import static com.mongodb.client.model.Sorts.ascending;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.DROPPED_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.KEY_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.MAX_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.MIN_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.NAMESPACE_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.OK_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.SHARD_FIELD;
+import static 
org.apache.flink.connector.mongodb.common.utils.MongoConstants.UUID_FIELD;
+
+/** A util class with some helper method for MongoDB commands. */
+@Internal
+public class MongoUtils {
+
+    public static final int UNAUTHORIZED_ERROR = 13;
+
+    public static final String COLL_STATS_COMMAND = "collStats";
+    public static final String SPLIT_VECTOR_COMMAND = "splitVector";
+    public static final String KEY_PATTERN_OPTION = "keyPattern";
+    public static final String MAX_CHUNK_SIZE_OPTION = "maxChunkSize";
+
+    public static final String CONFIG_DATABASE = "config";
+    public static final String COLLECTIONS_COLLECTION = "collections";
+    public static final String CHUNKS_COLLECTION = "chunks";
+
+    private MongoUtils() {}
+
+    public static BsonDocument collStats(MongoClient mongoClient, 
MongoNamespace namespace) {
+        BsonDocument collStatsCommand =
+                new BsonDocument(COLL_STATS_COMMAND, new 
BsonString(namespace.getCollectionName()));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(collStatsCommand, BsonDocument.class);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB) {
+        return splitVector(mongoClient, namespace, keyPattern, maxChunkSizeMB, 
null, null);
+    }
+
+    public static BsonDocument splitVector(
+            MongoClient mongoClient,
+            MongoNamespace namespace,
+            BsonDocument keyPattern,
+            int maxChunkSizeMB,
+            @Nullable BsonDocument min,
+            @Nullable BsonDocument max) {
+        BsonDocument splitVectorCommand =
+                new BsonDocument(SPLIT_VECTOR_COMMAND, new 
BsonString(namespace.getFullName()))
+                        .append(KEY_PATTERN_OPTION, keyPattern)
+                        .append(MAX_CHUNK_SIZE_OPTION, new 
BsonInt32(maxChunkSizeMB));
+        Optional.ofNullable(min).ifPresent(v -> 
splitVectorCommand.append(MIN_FIELD, v));
+        Optional.ofNullable(max).ifPresent(v -> 
splitVectorCommand.append(MAX_FIELD, v));
+        return mongoClient
+                .getDatabase(namespace.getDatabaseName())
+                .runCommand(splitVectorCommand, BsonDocument.class);
+    }
+
+    @Nullable

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



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

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

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

Reply via email to