godfreyhe commented on code in PR #20432:
URL: https://github.com/apache/flink/pull/20432#discussion_r939509904


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java:
##########
@@ -0,0 +1,510 @@
+/*
+ * 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.table.planner.plan.rules.logical;
+
+import org.apache.flink.table.api.TableException;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.rules.CoreRules;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.TransformationRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.ImmutableBeans;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.calcite.plan.RelOptUtil.conjunctions;
+
+/**
+ * Planner rule that pushes filters above and within a join node into the join 
node and/or its
+ * children nodes.
+ *
+ * <p>This rule is copied from {@link FilterJoinRule}.
+ *

Review Comment:
   I can add more comments



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java:
##########
@@ -0,0 +1,510 @@
+/*
+ * 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.table.planner.plan.rules.logical;
+
+import org.apache.flink.table.api.TableException;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinInfo;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.rules.CoreRules;
+import org.apache.calcite.rel.rules.FilterJoinRule;
+import org.apache.calcite.rel.rules.TransformationRule;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.ImmutableBeans;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.calcite.plan.RelOptUtil.conjunctions;
+
+/**
+ * Planner rule that pushes filters above and within a join node into the join 
node and/or its
+ * children nodes.
+ *
+ * <p>This rule is copied from {@link FilterJoinRule}.
+ *
+ * <p>For the above filter of inner/left/right join or the join condition of 
inner join, the
+ * predicate which field references are all from one side join condition can 
be pushed into another
+ * join side. Such as:
+ * <li>SELECT * FROM MyTable1 join MyTable2 ON a1 = a2 AND a1 = 2
+ * <li>SELECT * FROM MyTable1, MyTable2 WHERE a1 = a2 AND a1 = 2
+ */
+public abstract class FlinkFilterJoinRule<C extends 
FlinkFilterJoinRule.Config> extends RelRule<C>
+        implements TransformationRule {
+
+    public static final FlinkFilterIntoJoinRule FILTER_INTO_JOIN =
+            FlinkFilterIntoJoinRule.Config.DEFAULT.toRule();
+    public static final FlinkJoinConditionPushRule JOIN_CONDITION_PUSH =
+            FlinkJoinConditionPushRule.Config.DEFAULT.toRule();
+
+    /** Creates a FilterJoinRule. */
+    protected FlinkFilterJoinRule(C config) {
+        super(config);
+    }
+
+    // ~ Methods 
----------------------------------------------------------------
+
+    protected void perform(RelOptRuleCall call, Filter filter, Join join) {
+        final List<RexNode> joinFilters = 
RelOptUtil.conjunctions(join.getCondition());
+        final List<RexNode> origJoinFilters =
+                com.google.common.collect.ImmutableList.copyOf(joinFilters);
+
+        // If there is only the joinRel,
+        // make sure it does not match a cartesian product joinRel
+        // (with "true" condition), otherwise this rule will be applied
+        // again on the new cartesian product joinRel.
+        if (filter == null && joinFilters.isEmpty()) {
+            return;
+        }
+
+        final List<RexNode> aboveFilters =
+                filter != null ? getConjunctions(filter) : new ArrayList<>();
+        final com.google.common.collect.ImmutableList<RexNode> 
origAboveFilters =
+                com.google.common.collect.ImmutableList.copyOf(aboveFilters);
+
+        // Simplify Outer Joins
+        JoinRelType joinType = join.getJoinType();
+        if (config.isSmart()
+                && !origAboveFilters.isEmpty()
+                && join.getJoinType() != JoinRelType.INNER) {
+            joinType = RelOptUtil.simplifyJoin(join, origAboveFilters, 
joinType);
+        }
+
+        final List<RexNode> leftFilters = new ArrayList<>();
+        final List<RexNode> rightFilters = new ArrayList<>();
+
+        // TODO - add logic to derive additional filters.  E.g., from
+        // (t1.a = 1 AND t2.a = 2) OR (t1.b = 3 AND t2.b = 4), you can
+        // derive table filters:
+        // (t1.a = 1 OR t1.b = 3)
+        // (t2.a = 2 OR t2.b = 4)
+
+        // Try to push down above filters. These are typically where clause
+        // filters. They can be pushed down if they are not on the NULL
+        // generating side.
+        boolean filterPushed = false;
+        if (RelOptUtil.classifyFilters(
+                join,
+                aboveFilters,
+                joinType,
+                true,
+                !joinType.generatesNullsOnLeft(),
+                !joinType.generatesNullsOnRight(),
+                joinFilters,
+                leftFilters,
+                rightFilters)) {
+            filterPushed = true;
+        }

Review Comment:
   I do not change the copy code



-- 
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]

Reply via email to