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<RowData, BsonValue> keyExtractor; + + public MongoDynamicTableSink( + MongoConnectionOptions connectionOptions, + MongoWriteOptions writeOptions, + @Nullable Integer parallelism, + DataType physicalRowDataType, + SerializableFunction<RowData, BsonValue> 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
