You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/10/23 07:31:56 UTC

[GitHub] [flink] libenchao commented on a diff in pull request #20140: [Flink 16024][Connector][JDBC] Support FilterPushdown

libenchao commented on code in PR #20140:
URL: https://github.com/apache/flink/pull/20140#discussion_r1002654132


##########
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcFilterPushdownPreparedStatementVisitor.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.jdbc.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.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Visitor that convert Expression to ParameterizedPredicate. Return Optional.empty() if we cannot
+ * push down the filter.
+ */
+@Experimental
+public class JdbcFilterPushdownPreparedStatementVisitor
+        extends ExpressionDefaultVisitor<Optional<ParameterizedPredicate>> {
+
+    Function<String, String> quoteIdentifierFunction;
+
+    public static Set<Class<?>> supportedDataTypes;

Review Comment:
   These two fields can both be `private`?



##########
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java:
##########
@@ -159,8 +206,12 @@ public void applyProjection(int[][] projectedFields, DataType producedDataType)
 
     @Override
     public DynamicTableSource copy() {
-        return new JdbcDynamicTableSource(
-                options, readOptions, lookupMaxRetryTimes, cache, physicalRowDataType);
+        JdbcDynamicTableSource newSource =
+                new JdbcDynamicTableSource(
+                        options, readOptions, lookupMaxRetryTimes, cache, physicalRowDataType);
+        newSource.resolvedPredicates = this.resolvedPredicates;
+        newSource.pushdownParams = this.pushdownParams;

Review Comment:
   Make a deep copy for these two params cause they are mutable.



##########
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java:
##########
@@ -117,21 +137,48 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon
                         options.getTableName(),
                         DataType.getFieldNames(physicalRowDataType).toArray(new String[0]),
                         new String[0]);
+        final List<String> predicates = new ArrayList<String>();
+
         if (readOptions.getPartitionColumnName().isPresent()) {
             long lowerBound = readOptions.getPartitionLowerBound().get();
             long upperBound = readOptions.getPartitionUpperBound().get();
             int numPartitions = readOptions.getNumPartitions().get();
+
+            Serializable[][] allPushdownParams = replicatePushdownParamsForN(numPartitions);
+            JdbcParameterValuesProvider allParams =
+                    new CompositeJdbcParameterValuesProvider(
+                            new JdbcNumericBetweenParametersProvider(lowerBound, upperBound)
+                                    .ofBatchNum(numPartitions),
+                            new JdbcGenericParameterValuesProvider(allPushdownParams));
+
+            builder.setParametersProvider(allParams);
+
+            predicates.add(
+                    dialect.quoteIdentifier(readOptions.getPartitionColumnName().get())
+                            + " BETWEEN ? AND ?");
+        } else {
             builder.setParametersProvider(
-                    new JdbcNumericBetweenParametersProvider(lowerBound, upperBound)
-                            .ofBatchNum(numPartitions));
-            query +=
-                    " WHERE "
-                            + dialect.quoteIdentifier(readOptions.getPartitionColumnName().get())
-                            + " BETWEEN ? AND ?";
+                    new JdbcGenericParameterValuesProvider(replicatePushdownParamsForN(1)));
         }
+
+        predicates.addAll(this.resolvedPredicates);
+
+        if (predicates.size() > 0) {
+            String joinedConditions =
+                    predicates.stream()
+                            .map(pred -> String.format("(%s)", pred))
+                            .collect(Collectors.joining(" AND "));
+            query += " WHERE " + joinedConditions;
+        }
+
         if (limit >= 0) {
             query = String.format("%s %s", query, dialect.getLimitClause(limit));
         }
+
+        if (log.isDebugEnabled()) {
+            log.debug("Query generated for JDBC scan: " + query);
+        }

Review Comment:
   It's not necessary to do `if (log.isDebugEnabled())`?



##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java:
##########
@@ -264,6 +264,146 @@ void testLimit() throws Exception {
                 .containsAll(result);
     }
 
+    @Test
+    public void testFilter() throws Exception {

Review Comment:
   Could you also add a test in `JdbcTablePlanTest` like `JdbcTablePlanTest#testLimitPushDown`?



##########
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java:
##########
@@ -22,33 +22,51 @@
 import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
 import org.apache.flink.connector.jdbc.internal.options.JdbcConnectorOptions;
 import org.apache.flink.connector.jdbc.internal.options.JdbcReadOptions;
+import org.apache.flink.connector.jdbc.split.CompositeJdbcParameterValuesProvider;
+import org.apache.flink.connector.jdbc.split.JdbcGenericParameterValuesProvider;
 import org.apache.flink.connector.jdbc.split.JdbcNumericBetweenParametersProvider;
+import org.apache.flink.connector.jdbc.split.JdbcParameterValuesProvider;
 import org.apache.flink.table.connector.ChangelogMode;
 import org.apache.flink.table.connector.Projection;
 import org.apache.flink.table.connector.source.DynamicTableSource;
 import org.apache.flink.table.connector.source.InputFormatProvider;
 import org.apache.flink.table.connector.source.LookupTableSource;
 import org.apache.flink.table.connector.source.ScanTableSource;
+import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown;
 import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown;
 import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown;
 import org.apache.flink.table.connector.source.lookup.LookupFunctionProvider;
 import org.apache.flink.table.connector.source.lookup.PartialCachingLookupProvider;
 import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.expressions.CallExpression;
+import org.apache.flink.table.expressions.ResolvedExpression;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.logical.RowType;
 import org.apache.flink.util.Preconditions;
 
+import org.apache.commons.lang3.ArrayUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import javax.annotation.Nullable;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
 import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
 
 /** A {@link DynamicTableSource} for JDBC. */
 @Internal
 public class JdbcDynamicTableSource
         implements ScanTableSource,
                 LookupTableSource,
                 SupportsProjectionPushDown,
-                SupportsLimitPushDown {
+                SupportsLimitPushDown,
+                SupportsFilterPushDown {
+    private static Logger log = LoggerFactory.getLogger(JdbcDynamicTableSource.class);

Review Comment:
   `log` -> `LOG`



##########
flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcFilterPushdownPreparedStatementVisitorITCase.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.jdbc.table;
+
+import org.apache.flink.connector.jdbc.JdbcTestBase;
+import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
+import org.apache.flink.connector.jdbc.dialect.derby.DerbyDialectFactory;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableException;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.FunctionCatalog;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.expressions.ResolvedExpression;
+import org.apache.flink.table.expressions.resolver.ExpressionResolver;
+import org.apache.flink.table.planner.calcite.FlinkContext;
+import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
+import org.apache.flink.table.planner.calcite.FlinkTypeSystem;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.expressions.RexNodeExpression;
+import org.apache.flink.table.planner.plan.utils.RexNodeToExpressionConverter;
+import org.apache.flink.table.planner.runtime.utils.StreamTestSink;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.apache.calcite.rex.RexBuilder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+import java.util.TimeZone;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/** Test for {@link JdbcFilterPushdownPreparedStatementVisitor}. */
+public class JdbcFilterPushdownPreparedStatementVisitorITCase extends AbstractTestBase {

Review Comment:
   Why do you name this an `ITCase` instead of normal UT?



##########
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcFilterPushdownPreparedStatementVisitor.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.jdbc.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.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Visitor that convert Expression to ParameterizedPredicate. Return Optional.empty() if we cannot
+ * push down the filter.
+ */
+@Experimental
+public class JdbcFilterPushdownPreparedStatementVisitor
+        extends ExpressionDefaultVisitor<Optional<ParameterizedPredicate>> {
+
+    Function<String, String> quoteIdentifierFunction;
+
+    public static Set<Class<?>> supportedDataTypes;

Review Comment:
   And we usually use upper case for static field names, also make fields `final` as much as possible.



##########
flink-connectors/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcFilterPushdownPreparedStatementVisitor.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.jdbc.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.BigIntType;
+import org.apache.flink.table.types.logical.BooleanType;
+import org.apache.flink.table.types.logical.DateType;
+import org.apache.flink.table.types.logical.DecimalType;
+import org.apache.flink.table.types.logical.DoubleType;
+import org.apache.flink.table.types.logical.FloatType;
+import org.apache.flink.table.types.logical.IntType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.SmallIntType;
+import org.apache.flink.table.types.logical.TimestampType;
+import org.apache.flink.table.types.logical.VarCharType;
+
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.function.Supplier;
+
+/**
+ * Visitor that convert Expression to ParameterizedPredicate. Return Optional.empty() if we cannot
+ * push down the filter.
+ */
+@Experimental
+public class JdbcFilterPushdownPreparedStatementVisitor
+        extends ExpressionDefaultVisitor<Optional<ParameterizedPredicate>> {
+
+    Function<String, String> quoteIdentifierFunction;
+
+    public static Set<Class<?>> supportedDataTypes;
+
+    static {
+        supportedDataTypes = new HashSet<>();
+        supportedDataTypes.add(IntType.class);
+        supportedDataTypes.add(BigIntType.class);
+        supportedDataTypes.add(BooleanType.class);
+        supportedDataTypes.add(DecimalType.class);
+        supportedDataTypes.add(DoubleType.class);
+        supportedDataTypes.add(FloatType.class);
+        supportedDataTypes.add(SmallIntType.class);
+        supportedDataTypes.add(VarCharType.class);
+        supportedDataTypes.add(TimestampType.class);
+    }
+
+    public JdbcFilterPushdownPreparedStatementVisitor(
+            Function<String, String> quoteIdentifierFunction) {
+        this.quoteIdentifierFunction = quoteIdentifierFunction;
+    }
+
+    @Override
+    public Optional<ParameterizedPredicate> visit(CallExpression call) {
+        if (BuiltInFunctionDefinitions.EQUALS.equals(call.getFunctionDefinition())) {
+            return renderBinaryOperator("=", call.getResolvedChildren());

Review Comment:
   How about adding `<>` too since it's also a simple binary operator.



-- 
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-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org