xuyangzhong commented on code in PR #17: URL: https://github.com/apache/flink-connector-mongodb/pull/17#discussion_r1374396462
########## flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java: ########## @@ -178,6 +196,41 @@ public void applyProjection(int[][] projectedFields, DataType producedDataType) this.producedDataType = producedDataType; } + @Override + public Result applyFilters(List<ResolvedExpression> filters) { + List<ResolvedExpression> acceptedFilters = new ArrayList<>(); + List<ResolvedExpression> remainingFilters = new ArrayList<>(); + + List<Bson> mongoFilters = new ArrayList<>(); + for (ResolvedExpression filter : filters) { + BsonDocument simpleFilter = parseFilter(filter); + if (simpleFilter.isEmpty()) { + remainingFilters.add(filter); + } else { + acceptedFilters.add(filter); + mongoFilters.add(simpleFilter); + } + } + + if (!mongoFilters.isEmpty()) { + Bson mergedFilter = + mongoFilters.size() == 1 ? mongoFilters.get(0) : Filters.and(mongoFilters); + this.filter = mergedFilter.toBsonDocument(); + LOG.debug("Pushed down filters: {}", filter.toJson()); Review Comment: I prefer `LOG.info` here, WDYT? ########## flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java: ########## @@ -148,13 +163,16 @@ public ChangelogMode getChangelogMode() { @Override public DynamicTableSource copy() { - return new MongoDynamicTableSource( - connectionOptions, - readOptions, - lookupCache, - lookupMaxRetries, - lookupRetryIntervalMs, - producedDataType); + MongoDynamicTableSource newSource = + new MongoDynamicTableSource( + connectionOptions, + readOptions, + lookupCache, + lookupMaxRetries, + lookupRetryIntervalMs, + producedDataType); + newSource.filter = BsonDocument.parse(filter.toJson()); Review Comment: Why not use `newSource.filter = this.filter;` directly? ########## flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSource.java: ########## @@ -178,6 +196,41 @@ public void applyProjection(int[][] projectedFields, DataType producedDataType) this.producedDataType = producedDataType; } + @Override + public Result applyFilters(List<ResolvedExpression> filters) { + List<ResolvedExpression> acceptedFilters = new ArrayList<>(); Review Comment: I'm not sure if this feature needs to be compatible with the behavior before while users upgrade their jobs. If it is, maybe adding a config for it is better. ########## flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoFilterPushDownVisitor.java: ########## @@ -0,0 +1,251 @@ +/* + * 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.Experimental; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionDefaultVisitor; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.types.logical.LogicalType; + +import com.mongodb.client.model.Filters; +import org.bson.BsonBoolean; +import org.bson.BsonDateTime; +import org.bson.BsonDecimal128; +import org.bson.BsonDocument; +import org.bson.BsonDouble; +import org.bson.BsonInt32; +import org.bson.BsonInt64; +import org.bson.BsonNull; +import org.bson.BsonString; +import org.bson.BsonType; +import org.bson.BsonUndefined; +import org.bson.BsonValue; +import org.bson.conversions.Bson; +import org.bson.types.Decimal128; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.List; +import java.util.Optional; + +/** + * Visitor that convert Expression to Bson filter. Return {@link Filters#empty()} if we cannot push + * down the filter. + */ +@Experimental +public class MongoFilterPushDownVisitor extends ExpressionDefaultVisitor<BsonValue> { + + public static final MongoFilterPushDownVisitor INSTANCE = new MongoFilterPushDownVisitor(); + + private MongoFilterPushDownVisitor() { + // Singleton instance. + } + + @Override + public BsonDocument visit(CallExpression call) { + Bson filter = Filters.empty(); + if (BuiltInFunctionDefinitions.EQUALS.equals(call.getFunctionDefinition())) { Review Comment: nit: using `switch` instead of too many `if`, and return directly if matched. ########## flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/source/MongoSourceITCase.java: ########## @@ -76,7 +76,7 @@ /** IT cases for using Mongo Sink. */ @Testcontainers -public class MongoSourceITCase { +class MongoSourceITCase { Review Comment: I wonder why you remove the `public` here and in the other functions? ########## flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/table/MongoFilterPushDownVisitor.java: ########## @@ -0,0 +1,251 @@ +/* + * 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.Experimental; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.ExpressionDefaultVisitor; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.types.logical.LogicalType; + +import com.mongodb.client.model.Filters; +import org.bson.BsonBoolean; +import org.bson.BsonDateTime; +import org.bson.BsonDecimal128; +import org.bson.BsonDocument; +import org.bson.BsonDouble; +import org.bson.BsonInt32; +import org.bson.BsonInt64; +import org.bson.BsonNull; +import org.bson.BsonString; +import org.bson.BsonType; +import org.bson.BsonUndefined; +import org.bson.BsonValue; +import org.bson.conversions.Bson; +import org.bson.types.Decimal128; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.List; +import java.util.Optional; + +/** + * Visitor that convert Expression to Bson filter. Return {@link Filters#empty()} if we cannot push + * down the filter. + */ +@Experimental +public class MongoFilterPushDownVisitor extends ExpressionDefaultVisitor<BsonValue> { + + public static final MongoFilterPushDownVisitor INSTANCE = new MongoFilterPushDownVisitor(); + + private MongoFilterPushDownVisitor() { + // Singleton instance. + } + + @Override + public BsonDocument visit(CallExpression call) { + Bson filter = Filters.empty(); + if (BuiltInFunctionDefinitions.EQUALS.equals(call.getFunctionDefinition())) { + filter = renderBinaryComparisonOperator("$eq", call.getResolvedChildren()); + } + if (BuiltInFunctionDefinitions.LESS_THAN.equals(call.getFunctionDefinition())) { + filter = renderBinaryComparisonOperator("$lt", call.getResolvedChildren()); + } + if (BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL.equals(call.getFunctionDefinition())) { + filter = renderBinaryComparisonOperator("$lte", call.getResolvedChildren()); + } + if (BuiltInFunctionDefinitions.GREATER_THAN.equals(call.getFunctionDefinition())) { + filter = renderBinaryComparisonOperator("$gt", call.getResolvedChildren()); + } + if (BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL.equals(call.getFunctionDefinition())) { + filter = renderBinaryComparisonOperator("$gte", call.getResolvedChildren()); + } + if (BuiltInFunctionDefinitions.NOT_EQUALS.equals(call.getFunctionDefinition())) { + filter = renderBinaryComparisonOperator("$ne", call.getResolvedChildren()); + } + if (BuiltInFunctionDefinitions.IS_NULL.equals(call.getFunctionDefinition())) { + filter = + renderUnaryComparisonOperator( + "$eq", call.getResolvedChildren().get(0), BsonNull.VALUE); + } + if (BuiltInFunctionDefinitions.IS_NOT_NULL.equals(call.getFunctionDefinition())) { + filter = + renderUnaryComparisonOperator( + "$ne", call.getResolvedChildren().get(0), BsonNull.VALUE); + } + if (BuiltInFunctionDefinitions.OR.equals(call.getFunctionDefinition())) { + filter = renderLogicalOperator("$or", call.getResolvedChildren()); + } + if (BuiltInFunctionDefinitions.AND.equals(call.getFunctionDefinition())) { + filter = renderLogicalOperator("$and", call.getResolvedChildren()); + } + return filter.toBsonDocument(); + } + + private Bson renderBinaryComparisonOperator( + String operator, List<ResolvedExpression> expressions) { + Optional<FieldReferenceExpression> fieldReferenceExpr = Review Comment: Just curious, currently `f1 = f2` is not supported while `f1` and `f2` are both source fields? ########## flink-connector-mongodb/src/test/java/org/apache/flink/connector/mongodb/table/MongoDynamicTableSourceITCase.java: ########## @@ -319,12 +361,13 @@ private static String createTestDDl(Map<String, String> extraOptions) { "CREATE TABLE mongo_source", "(", " _id BIGINT,", + " f0 STRING,", " f1 STRING,", " f2 BOOLEAN,", " f3 BINARY,", " f4 INTEGER,", - " f5 TIMESTAMP_LTZ(6),", - " f6 TIMESTAMP_LTZ(3),", + " f5 TIMESTAMP_LTZ(3),", Review Comment: I'm not sure about these changes. These changes seem to be changed by refacting test data, right? -- 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