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


##########
flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/sink/writer/MongoWriter.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.mongodb.sink.writer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.api.common.functions.util.ListCollector;
+import org.apache.flink.api.common.operators.MailboxExecutor;
+import org.apache.flink.api.common.serialization.SerializationSchema;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.connector.mongodb.common.config.MongoConnectionOptions;
+import org.apache.flink.connector.mongodb.sink.config.MongoWriteOptions;
+import 
org.apache.flink.connector.mongodb.sink.writer.context.DefaultMongoSinkContext;
+import org.apache.flink.connector.mongodb.sink.writer.context.MongoSinkContext;
+import 
org.apache.flink.connector.mongodb.sink.writer.serializer.MongoSerializationSchema;
+import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
+import org.apache.flink.runtime.operators.util.metrics.CountingCollector;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import com.mongodb.MongoException;
+import com.mongodb.client.MongoClient;
+import com.mongodb.client.MongoClients;
+import com.mongodb.client.model.WriteModel;
+import org.bson.BsonDocument;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This class is responsible for writing records to a MongoDB collection.
+ *
+ * @param <IN> The type of the input elements.
+ */
+@Internal
+public class MongoWriter<IN> implements SinkWriter<IN> {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(MongoWriter.class);
+
+    private final MongoConnectionOptions connectionOptions;
+    private final MongoWriteOptions writeOptions;
+    private final MongoSerializationSchema<IN> serializationSchema;
+    private final MongoSinkContext sinkContext;
+    private final MailboxExecutor mailboxExecutor;
+    private final boolean flushOnCheckpoint;
+    private final List<WriteModel<BsonDocument>> bulkRequests = new 
ArrayList<>();
+    private final Collector<WriteModel<BsonDocument>> collector;
+    private final MongoClient mongoClient;
+
+    private boolean checkpointInProgress = false;
+    private volatile long lastSendTime = 0L;
+    private volatile long ackTime = Long.MAX_VALUE;
+
+    public MongoWriter(
+            MongoConnectionOptions connectionOptions,
+            MongoWriteOptions writeOptions,
+            boolean flushOnCheckpoint,
+            Sink.InitContext initContext,
+            MongoSerializationSchema<IN> serializationSchema) {
+        this.connectionOptions = checkNotNull(connectionOptions);
+        this.writeOptions = checkNotNull(writeOptions);
+        this.serializationSchema = checkNotNull(serializationSchema);
+        this.flushOnCheckpoint = flushOnCheckpoint;
+
+        checkNotNull(initContext);
+        this.mailboxExecutor = checkNotNull(initContext.getMailboxExecutor());
+
+        SinkWriterMetricGroup metricGroup = 
checkNotNull(initContext.metricGroup());
+        metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime);
+
+        this.collector =
+                new CountingCollector<>(
+                        new ListCollector<>(this.bulkRequests),
+                        metricGroup.getNumRecordsSendCounter());
+
+        // Initialize the serialization schema.
+        this.sinkContext = new DefaultMongoSinkContext(initContext, 
writeOptions);
+        try {
+            SerializationSchema.InitializationContext initializationContext =
+                    initContext.asSerializationSchemaInitializationContext();
+            serializationSchema.open(initializationContext, sinkContext, 
writeOptions);
+        } catch (Exception e) {
+            throw new FlinkRuntimeException("Failed to open the MongoEmitter", 
e);
+        }
+
+        // Initialize the mongo client.
+        this.mongoClient = MongoClients.create(connectionOptions.getUri());
+    }
+
+    @Override
+    public void write(IN element, Context context) throws IOException, 
InterruptedException {
+        // do not allow new bulk writes until all actions are flushed
+        while (checkpointInProgress) {
+            mailboxExecutor.yield();
+        }
+        collector.collect(serializationSchema.serialize(element, sinkContext));
+        if (isOverMaxBatchSizeLimit() || isOverMaxBatchIntervalLimit()) {
+            doBulkWrite();
+        }
+    }
+
+    @Override
+    public void flush(boolean endOfInput) throws IOException {
+        checkpointInProgress = true;
+        while (!bulkRequests.isEmpty() && (flushOnCheckpoint || endOfInput)) {
+            doBulkWrite();
+        }
+        checkpointInProgress = false;
+    }
+
+    @Override
+    public void close() {
+        mongoClient.close();
+    }
+
+    @VisibleForTesting
+    void doBulkWrite() throws IOException {
+        if (bulkRequests.isEmpty()) {
+            // no records to write
+            return;
+        }
+
+        int maxRetries = writeOptions.getMaxRetries();

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



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

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



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to