zentol commented on code in PR #1: URL: https://github.com/apache/flink-connector-mongodb/pull/1#discussion_r1030405674
########## 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 + public static BsonDocument readCollectionMetadata( + MongoClient mongoClient, MongoNamespace namespace) { + MongoCollection<BsonDocument> collection = + mongoClient + .getDatabase(CONFIG_DATABASE) + .getCollection(COLLECTIONS_COLLECTION) + .withDocumentClass(BsonDocument.class); + + return collection + .find(eq(ID_FIELD, namespace.getFullName())) + .projection(include(ID_FIELD, UUID_FIELD, DROPPED_FIELD, KEY_FIELD)) + .first(); + } + + public static boolean isValidShardedCollection(BsonDocument collectionMetadata) { + return collectionMetadata != null + && !collectionMetadata.getBoolean(DROPPED_FIELD, BsonBoolean.FALSE).getValue(); + } + + public static List<BsonDocument> readChunks( + MongoClient mongoClient, BsonDocument collectionMetadata) { + MongoCollection<BsonDocument> chunks = + mongoClient + .getDatabase(CONFIG_DATABASE) + .getCollection(CHUNKS_COLLECTION) + .withDocumentClass(BsonDocument.class); + + Bson filter = + or( + new BsonDocument(NAMESPACE_FIELD, collectionMetadata.get(ID_FIELD)), + // MongoDB 4.9.0 removed ns field of config.chunks collection, using Review Comment: My question was less aimed at what the driver supports, but rather with MongoDB server versions are actively supported by the project. For example, if 3.x is no longer supported by MongoDB (irrespective of whether the driver supports it), then we should consider also not supporting 3.x. The immediate benefit would be that (I think) we'd no longer have to bother with the `dropped` flag at all. According to https://www.mongodb.com/support-policy/lifecycles 3.6 already reached EOL in April 20**21**. ########## flink-connector-mongodb-e2e-tests/src/test/java/org/apache/flink/tests/util/mongodb/MongoE2ECase.java: ########## @@ -134,13 +134,13 @@ private List<Document> readAllBackupOrders(MongoDatabase db, int expect) throws return backupOrders; } - private List<Document> mockOrders(int ordersCount) { Review Comment: I'm not sure why you changed this; here's no need for it. There is are `ElementsOf` variants of the assertj matchers that accept an iterable. ########## flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoKeyExtractor.java: ########## @@ -0,0 +1,139 @@ +/* + * 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.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"; + + 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> reversedId = resolvedSchema.getColumn(RESERVED_ID); + + // It behaves as append-only when no primary key is declared and no reversed _id is present. + // We use anonymous classes instead of lambdas for a reason here. It is + // necessary because the maven shade plugin cannot relocate classes in SerializedLambdas + // (MSHADE-260). + if (!primaryKey.isPresent() && !reversedId.isPresent()) { + return new SerializableFunction<RowData, BsonValue>() { + private static final long serialVersionUID = 1L; + + @Override + public BsonValue apply(RowData rowData) { + return null; + } + }; + } + + if (reversedId.isPresent()) { + // Primary key should be declared as (_id) when the mongo reversed _id is present. + if (!primaryKey.isPresent() + || isCompoundPrimaryKey(primaryKeyIndexes) + || !primaryKeyContainsReversedId(primaryKey.get())) { + throw new IllegalArgumentException( + "The primary key should be declared as (_id) when mongo reversed _id field is present"); Review Comment: That's an interesting question. We don't seem to handle this case in a special way in the ES connector; we use the `PRIMARY KEY` as the id and insert the document as is with it's _id_ field; the final behavior will depend on Elasticsearch. I don't really like that behavior because as you said it is ambiguous; we should fail early if the schema contains an _id_ field and it's not the sole primary key. My comment was rather about the error message. You are explicitly suggesting `(_id)` as an alternative in this case; but is that really the best option? Maybe they just made a mistake w.r.t. keys. Maybe we shouldn't even give such suggestions, and rather just describe the problem of ambiguous keys being used due to the presence of an `_id` field ########## flink-connector-mongodb-e2e-tests/pom.xml: ########## @@ -0,0 +1,156 @@ +<?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-connector-mongodb-e2e-tests</artifactId> + <name>Flink : E2E Tests : MongoDB</name> + + <dependencies> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-test-utils</artifactId> + <version>${flink.version}</version> + <scope>test</scope> + </dependency> + + <!-- Use fat jar so we don't need to create a user-jar. --> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-sql-connector-mongodb</artifactId> + <version>${project.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.mongodb</groupId> + <artifactId>mongodb-driver-sync</artifactId> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.testcontainers</groupId> + <artifactId>mongodb</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + + <profiles> + <profile> + <id>run-end-to-end-tests</id> + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-surefire-plugin</artifactId> + <executions> + <execution> + <id>end-to-end-tests</id> + <phase>integration-test</phase> + <goals> + <goal>test</goal> + </goals> + <configuration> + <includes> + <include>**/*.*</include> + </includes> + <forkCount>1</forkCount> Review Comment: The comment should only be removed if this line (you know, the one it documents) is removed (which it should be!) ########## flink-connector-mongodb-e2e-tests/pom.xml: ########## @@ -0,0 +1,117 @@ +<?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-connector-mongodb-e2e-tests</artifactId> + <name>Flink : E2E Tests : MongoDB</name> + + <dependencies> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-connector-test-utils</artifactId> + <version>${flink.version}</version> + <scope>test</scope> + </dependency> + + <!--using mongodb shade jar to execute end-to-end test--> + <dependency> + <groupId>org.apache.flink</groupId> + <artifactId>flink-sql-connector-mongodb</artifactId> + <version>${project.version}</version> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.mongodb</groupId> + <artifactId>mongodb-driver-sync</artifactId> + <scope>test</scope> + </dependency> + + <dependency> + <groupId>org.testcontainers</groupId> + <artifactId>mongodb</artifactId> + <scope>test</scope> + </dependency> + </dependencies> + + <build> + <plugins> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-deploy-plugin</artifactId> + <configuration> + <skip>true</skip> + </configuration> + </plugin> + <plugin> + <groupId>org.apache.maven.plugins</groupId> + <artifactId>maven-surefire-plugin</artifactId> + <configuration> + <skip>true</skip> + </configuration> + </plugin> Review Comment: This needs to be setup similarly to https://github.com/apache/flink-connector-elasticsearch/blob/main/flink-connector-elasticsearch-e2e-tests/pom.xml#L44; a dedicated profile that activates the test. (This is because these tests need some prep work beforehand, specifically downloading a Flink binary) -- 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]
