zentol commented on code in PR #1: URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1030553397
########## flink-sql-connector-mongodb/pom.xml: ########## @@ -0,0 +1,93 @@ +<?xml version="1.0" encoding="UTF-8"?> +<!-- +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. +--> +<project + xsi:schemaLocation="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"> + + <modelVersion>4.0.0</modelVersion> + + <parent> + <artifactId>flink-connector-mongodb-parent</artifactId> + <groupId>org.apache.flink</groupId> + <version>1.0-SNAPSHOT</version> + </parent> + + <artifactId>flink-sql-connector-mongodb</artifactId> + <name>Flink : Connectors : SQL : MongoDB</name> + + <packaging>jar</packaging> + + <properties> + <japicmp.skip>true</japicmp.skip> + </properties> + + <dependencies> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-mongodb</artifactId> + <version>${project.version}</version> + </dependency> + </dependencies> + + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-shade-plugin</artifactId> + <executions> + <execution> + <id>shade-flink</id> + <phase>package</phase> + <goals> + <goal>shade</goal> + </goals> + <configuration> + <artifactSet> + <includes> + <include>org.apache.flink:flink-connector-mongodb</include> Review Comment: To ease maintenance I'd set this to `*:*` to include everything the connector needs. ########## flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializerTest.java: ########## @@ -0,0 +1,81 @@ +/* + * 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.connector.mongodb.source.split.MongoScanSourceSplit; + +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonMaxKey; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT; +import static org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumStateSerializer.INSTANCE; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotSame; + +/** Unit tests for {@link MongoSourceEnumStateSerializer}. */ +public class MongoSourceEnumStateSerializerTest { + + @Test + void serializeAndDeserializeMongoSourceEnumState() throws Exception { + boolean initialized = false; + List<String> remainingCollections = Arrays.asList("db.remains0", "db.remains1"); + List<String> alreadyProcessedCollections = Arrays.asList("db.processed0", "db.processed1"); + List<MongoScanSourceSplit> remainingScanSplits = + Arrays.asList(createSourceSplit(0), createSourceSplit(1)); + + Map<String, MongoScanSourceSplit> assignedScanSplits = + Collections.singletonMap("split2", createSourceSplit(2)); + + MongoSourceEnumState state = + new MongoSourceEnumState( + remainingCollections, + alreadyProcessedCollections, + remainingScanSplits, + assignedScanSplits, + initialized); + + byte[] bytes = INSTANCE.serialize(state); + MongoSourceEnumState state1 = INSTANCE.deserialize(INSTANCE.getVersion(), bytes); + + assertEquals(state.getRemainingCollections(), state1.getRemainingCollections()); + assertEquals( + state.getAlreadyProcessedCollections(), state1.getAlreadyProcessedCollections()); + assertEquals(state.getRemainingScanSplits(), state1.getRemainingScanSplits()); + assertEquals(state.getAssignedScanSplits(), state1.getAssignedScanSplits()); + assertEquals(state.isInitialized(), state1.isInitialized()); + + assertNotSame(state, state1); + } + + private MongoScanSourceSplit createSourceSplit(int index) { Review Comment: ```suggestion private static MongoScanSourceSplit createSourceSplit(int index) { ``` ########## flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/common/utils/MongoSerdeUtilsTest.java: ########## @@ -0,0 +1,92 @@ +/* + * 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.junit.jupiter.api.Test; + +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.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit tests for {@link MongoSerdeUtils}. */ +public class MongoSerdeUtilsTest { Review Comment: ```suggestion class MongoSerdeUtilsTest { ``` Test classes/methods no longer need to be public. ########## 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<RowData, BsonValue> { + + 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)) { Review Comment: May nee changes depending on other discussions about supported types. Just marking this here as a reminder. ########## flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/MongoConfiguration.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy; +import org.apache.flink.table.connector.source.lookup.LookupOptions; + +import javax.annotation.Nullable; + +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.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; + +/** MongoDB configuration. */ +@PublicEvolving +public class MongoConfiguration implements Serializable { Review Comment: Does not have to be serializable afaict. ########## 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<RowData, BsonValue> { + + 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<RowData, BsonValue> createKeyExtractor( + ResolvedSchema resolvedSchema) { + + Optional<UniqueConstraint> primaryKey = resolvedSchema.getPrimaryKey(); + int[] primaryKeyIndexes = resolvedSchema.getPrimaryKeyIndexes(); + Optional<Column> reservedId = resolvedSchema.getColumn(RESERVED_ID); + + // Primary key is not declared and reserved _id is not present. Review Comment: Curious whether other connectors have any logic to silently use a field as a key. Sounds a bit iffy to me. :shrug: Seems that the ES connector would generate a random id in this case (because of how the client works). ########## 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: Is this something that mongodb users are expected to know? Maybe we should document these. ########## flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/common/utils/MongoSerdeUtilsTest.java: ########## @@ -0,0 +1,92 @@ +/* + * 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.junit.jupiter.api.Test; + +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.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit tests for {@link MongoSerdeUtils}. */ +public class MongoSerdeUtilsTest { + + @Test + public void testSerializeList() throws IOException { + List<String> expected = Arrays.asList("config.collections", "config.chunks"); + + byte[] serialized = serializeList(expected); + List<String> deserialized = deserializeList(serialized); + + assertThat(deserialized).isEqualTo(expected); + } + + @Test + public void testSerializeMap() throws IOException { + Map<String, String> expected = new HashMap<>(); + expected.put("k0", "v0"); + expected.put("k1", "v1"); + expected.put("k2", "v2"); + + byte[] serialized = serializeMap(expected); + + Map<String, String> deserialized = deserializeMap(serialized); + + assertThat(deserialized).isEqualTo(expected); + } + + private byte[] serializeList(List<String> list) throws IOException { Review Comment: these methods should be static ########## flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/enumerator/MongoSourceEnumStateSerializerTest.java: ########## @@ -0,0 +1,81 @@ +/* + * 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.connector.mongodb.source.split.MongoScanSourceSplit; + +import org.bson.BsonDocument; +import org.bson.BsonInt32; +import org.bson.BsonMaxKey; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.connector.mongodb.common.utils.MongoConstants.ID_HINT; +import static org.apache.flink.connector.mongodb.source.enumerator.MongoSourceEnumStateSerializer.INSTANCE; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotSame; + +/** Unit tests for {@link MongoSourceEnumStateSerializer}. */ +public class MongoSourceEnumStateSerializerTest { + + @Test + void serializeAndDeserializeMongoSourceEnumState() throws Exception { + boolean initialized = false; + List<String> remainingCollections = Arrays.asList("db.remains0", "db.remains1"); + List<String> alreadyProcessedCollections = Arrays.asList("db.processed0", "db.processed1"); + List<MongoScanSourceSplit> remainingScanSplits = + Arrays.asList(createSourceSplit(0), createSourceSplit(1)); + + Map<String, MongoScanSourceSplit> assignedScanSplits = + Collections.singletonMap("split2", createSourceSplit(2)); + + MongoSourceEnumState state = + new MongoSourceEnumState( + remainingCollections, + alreadyProcessedCollections, + remainingScanSplits, + assignedScanSplits, + initialized); + + byte[] bytes = INSTANCE.serialize(state); + MongoSourceEnumState state1 = INSTANCE.deserialize(INSTANCE.getVersion(), bytes); + + assertEquals(state.getRemainingCollections(), state1.getRemainingCollections()); + assertEquals( + state.getAlreadyProcessedCollections(), state1.getAlreadyProcessedCollections()); + assertEquals(state.getRemainingScanSplits(), state1.getRemainingScanSplits()); + assertEquals(state.getAssignedScanSplits(), state1.getAssignedScanSplits()); + assertEquals(state.isInitialized(), state1.isInitialized()); + + assertNotSame(state, state1); + } + + private MongoScanSourceSplit createSourceSplit(int index) { + return new MongoScanSourceSplit( + "split" + index, + "db", + "coll", + new BsonDocument("_id", new BsonInt32(index)), + new BsonDocument("_id", new BsonMaxKey()), Review Comment: ```suggestion new BsonDocument("_id", MongoConstants.BSON_MAX_KEY), ``` ########## 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<RowData, BsonValue> { + + 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]); Review Comment: Seems inefficient to recreate this getter for each value. The `FieldGetter` is supposed to be serializable, so it should be possible to do this once in the constructor (or initialize it lazily otherwise). ########## 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<MongoScanSourceSplit> 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: According to the `SupportsLimitPushDown` javadocs the limiting can be done on a best-effort basis. So for every split we could emit up to N elements, and have the Table runtime deal with the rest. ########## flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/config/MongoConfiguration.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.config; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.mongodb.source.enumerator.splitter.PartitionStrategy; +import org.apache.flink.table.connector.source.lookup.LookupOptions; + +import javax.annotation.Nullable; + +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.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; + +/** MongoDB configuration. */ +@PublicEvolving Review Comment: ```suggestion @Internal ``` This isn't user-facing is it? -- 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]
