You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by go...@apache.org on 2022/08/09 02:26:56 UTC

[flink] branch master updated: [FLINK-28753][table-planner] Improve FilterIntoJoinRule which could push some predicates to another side

This is an automated email from the ASF dual-hosted git repository.

godfrey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 0e6e4198ad8 [FLINK-28753][table-planner] Improve FilterIntoJoinRule which could push some predicates to another side
0e6e4198ad8 is described below

commit 0e6e4198ad84227c20e2c61c2dd8b0616324aa31
Author: godfreyhe <go...@163.com>
AuthorDate: Wed Aug 3 15:26:32 2022 +0800

    [FLINK-28753][table-planner] Improve FilterIntoJoinRule which could push some predicates to another side
    
    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
    
    This closes #20432
---
 .../plan/rules/logical/FlinkFilterJoinRule.java    | 510 +++++++++++++++++
 .../planner/plan/rules/FlinkBatchRuleSets.scala    |   4 +-
 .../planner/plan/rules/FlinkStreamRuleSets.scala   |   4 +-
 .../{joinhint => hints}/BroadcastJoinHintTest.java |   2 +-
 .../join/{joinhint => hints}/JoinHintTestBase.java |   2 +-
 .../{joinhint => hints}/NestLoopJoinHintTest.java  |   2 +-
 .../ShuffleHashJoinHintTest.java                   |   2 +-
 .../ShuffleMergeJoinHintTest.java                  |   2 +-
 .../optimize/ClearQueryBlockAliasResolverTest.java |   2 +-
 .../plan/optimize/JoinHintResolverTest.java        |   2 +-
 .../rules/logical/FlinkFilterJoinRuleTest.java     | 313 ++++++++++
 .../planner/plan/batch/sql/DeadlockBreakupTest.xml |  17 +-
 .../plan/batch/sql/DynamicFilteringTest.xml        |   8 +-
 .../plan/batch/sql/MultipleInputCreationTest.xml   |  18 +-
 .../planner/plan/batch/sql/SubplanReuseTest.xml    |  61 +-
 .../plan/batch/sql/join/BroadcastHashJoinTest.xml  | 297 +++++++++-
 .../plan/batch/sql/join/NestedLoopJoinTest.xml     | 319 +++++++++--
 .../plan/batch/sql/join/ShuffledHashJoinTest.xml   | 404 ++++++++++---
 .../plan/batch/sql/join/SortMergeJoinTest.xml      | 268 ++++++++-
 .../{joinhint => hints}/BroadcastJoinHintTest.xml  |  37 +-
 .../{joinhint => hints}/NestLoopJoinHintTest.xml   |  37 +-
 .../ShuffleHashJoinHintTest.xml                    |  37 +-
 .../ShuffleMergeJoinHintTest.xml                   |  37 +-
 .../plan/rules/logical/FlinkFilterJoinRuleTest.xml | 637 ++++++++++++++++++---
 .../planner/plan/stream/sql/SubplanReuseTest.xml   |  33 +-
 .../planner/plan/stream/sql/join/JoinTest.xml      | 364 ++++++++++--
 .../batch/sql/join/BroadcastHashJoinTest.scala     |   1 -
 .../planner/plan/batch/sql/join/JoinTestBase.scala |  66 +++
 .../plan/batch/sql/join/ShuffledHashJoinTest.scala |   4 +-
 .../rules/logical/FlinkFilterJoinRuleTest.scala    | 158 -----
 .../planner/plan/stream/sql/join/JoinTest.scala    |  73 ++-
 .../runtime/batch/sql/join/JoinITCase.scala        | 140 +++++
 .../planner/runtime/stream/sql/JoinITCase.scala    | 179 ++++++
 33 files changed, 3452 insertions(+), 588 deletions(-)

diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRule.java
new file mode 100644
index 00000000000..542db2ee58a
--- /dev/null
+++ b/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>Different from {@link FilterJoinRule}, this rule can handle more cases: 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;
+        }
+
+        // Move join filters up if needed
+        validateJoinFilters(aboveFilters, joinFilters, join, joinType);
+
+        // If no filter got pushed after validate, reset filterPushed flag
+        if (leftFilters.isEmpty()
+                && rightFilters.isEmpty()
+                && joinFilters.size() == origJoinFilters.size()
+                && aboveFilters.size() == origAboveFilters.size()) {
+            if (com.google.common.collect.Sets.newHashSet(joinFilters)
+                    .equals(com.google.common.collect.Sets.newHashSet(origJoinFilters))) {
+                filterPushed = false;
+            }
+        }
+
+        // Try to push down filters in ON clause. A ON clause filter can only be
+        // pushed down if it does not affect the non-matching set, i.e. it is
+        // not on the side which is preserved.
+
+        // Anti-join on conditions can not be pushed into left or right, e.g. for plan:
+        //
+        //     Join(condition=[AND(cond1, $2)], joinType=[anti])
+        //     :  - prj(f0=[$0], f1=[$1], f2=[$2])
+        //     :  - prj(f0=[$0])
+        //
+        // The semantic would change if join condition $2 is pushed into left,
+        // that is, the result set may be smaller. The right can not be pushed
+        // into for the same reason.
+        if (joinType != JoinRelType.ANTI
+                && RelOptUtil.classifyFilters(
+                        join,
+                        joinFilters,
+                        joinType,
+                        false,
+                        !joinType.generatesNullsOnRight(),
+                        !joinType.generatesNullsOnLeft(),
+                        joinFilters,
+                        leftFilters,
+                        rightFilters)) {
+            filterPushed = true;
+        }
+
+        // if nothing actually got pushed and there is nothing leftover,
+        // then this rule is a no-op
+        if ((!filterPushed && joinType == join.getJoinType())
+                || (joinFilters.isEmpty() && leftFilters.isEmpty() && rightFilters.isEmpty())) {
+            return;
+        }
+
+        final RexBuilder rexBuilder = join.getCluster().getRexBuilder();
+        // create the new join node referencing the new children and
+        // containing its new join filters (if there are any)
+        final com.google.common.collect.ImmutableList<RelDataType> fieldTypes =
+                com.google.common.collect.ImmutableList.<RelDataType>builder()
+                        .addAll(RelOptUtil.getFieldTypeList(join.getLeft().getRowType()))
+                        .addAll(RelOptUtil.getFieldTypeList(join.getRight().getRowType()))
+                        .build();
+        final RexNode joinFilter =
+                RexUtil.composeConjunction(
+                        rexBuilder, RexUtil.fixUp(rexBuilder, joinFilters, fieldTypes));
+
+        // push above filters to another side for INNER, LEFT, RIGHT join
+        pushFiltersToAnotherSide(
+                join,
+                joinType,
+                origAboveFilters,
+                joinFilter,
+                leftFilters,
+                rightFilters,
+                Arrays.asList(JoinRelType.INNER, JoinRelType.LEFT, JoinRelType.RIGHT));
+
+        // push join filters to another side for INNER join
+        pushFiltersToAnotherSide(
+                join,
+                joinType,
+                origJoinFilters,
+                null, // do not derive JoinInfo
+                leftFilters,
+                rightFilters,
+                Collections.singletonList(JoinRelType.INNER));
+
+        // create Filters on top of the children if any filters were
+        // pushed to them
+        final RelBuilder relBuilder = call.builder();
+        final RelNode leftRel = relBuilder.push(join.getLeft()).filter(leftFilters).build();
+        final RelNode rightRel = relBuilder.push(join.getRight()).filter(rightFilters).build();
+
+        // If nothing actually got pushed and there is nothing leftover,
+        // then this rule is a no-op
+        if (joinFilter.isAlwaysTrue()
+                && leftFilters.isEmpty()
+                && rightFilters.isEmpty()
+                && joinType == join.getJoinType()) {
+            return;
+        }
+
+        RelNode newJoinRel =
+                join.copy(
+                        join.getTraitSet(),
+                        joinFilter,
+                        leftRel,
+                        rightRel,
+                        joinType,
+                        join.isSemiJoinDone());
+        call.getPlanner().onCopy(join, newJoinRel);
+        if (!leftFilters.isEmpty()) {
+            call.getPlanner().onCopy(filter, leftRel);
+        }
+        if (!rightFilters.isEmpty()) {
+            call.getPlanner().onCopy(filter, rightRel);
+        }
+
+        relBuilder.push(newJoinRel);
+
+        // Create a project on top of the join if some of the columns have become
+        // NOT NULL due to the join-type getting stricter.
+        relBuilder.convert(join.getRowType(), false);
+
+        // create a FilterRel on top of the join if needed
+        relBuilder.filter(
+                RexUtil.fixUp(
+                        rexBuilder,
+                        aboveFilters,
+                        RelOptUtil.getFieldTypeList(relBuilder.peek().getRowType())));
+        call.transformTo(relBuilder.build());
+    }
+
+    /**
+     * Get conjunctions of filter's condition but with collapsed {@code IS NOT DISTINCT FROM}
+     * expressions if needed.
+     *
+     * @param filter filter containing condition
+     * @return condition conjunctions with collapsed {@code IS NOT DISTINCT FROM} expressions if any
+     * @see RelOptUtil#conjunctions(RexNode)
+     */
+    private List<RexNode> getConjunctions(Filter filter) {
+        List<RexNode> conjunctions = conjunctions(filter.getCondition());
+        RexBuilder rexBuilder = filter.getCluster().getRexBuilder();
+        for (int i = 0; i < conjunctions.size(); i++) {
+            RexNode node = conjunctions.get(i);
+            if (node instanceof RexCall) {
+                conjunctions.set(
+                        i,
+                        RelOptUtil.collapseExpandedIsNotDistinctFromExpr(
+                                (RexCall) node, rexBuilder));
+            }
+        }
+        return conjunctions;
+    }
+
+    /**
+     * Validates that target execution framework can satisfy join filters.
+     *
+     * <p>If the join filter cannot be satisfied (for example, if it is {@code l.c1 > r.c2} and the
+     * join only supports equi-join), removes the filter from {@code joinFilters} and adds it to
+     * {@code aboveFilters}.
+     *
+     * <p>The default implementation does nothing; i.e. the join can handle all conditions.
+     *
+     * @param aboveFilters Filter above Join
+     * @param joinFilters Filters in join condition
+     * @param join Join
+     * @param joinType JoinRelType could be different from type in Join due to outer join
+     *     simplification.
+     */
+    protected void validateJoinFilters(
+            List<RexNode> aboveFilters,
+            List<RexNode> joinFilters,
+            Join join,
+            JoinRelType joinType) {
+        final Iterator<RexNode> filterIter = joinFilters.iterator();
+        while (filterIter.hasNext()) {
+            RexNode exp = filterIter.next();
+            // Do not pull up filter conditions for semi/anti join.
+            if (!config.getPredicate().apply(join, joinType, exp) && joinType.projectsRight()) {
+                aboveFilters.add(exp);
+                filterIter.remove();
+            }
+        }
+    }
+
+    private void pushFiltersToAnotherSide(
+            Join joinRel,
+            JoinRelType joinType,
+            List<RexNode> filtersToPush,
+            @Nullable RexNode joinFilter,
+            List<RexNode> leftFilters,
+            List<RexNode> rightFilters,
+            List<JoinRelType> expectedJoinTypes) {
+        if (filtersToPush.isEmpty() || !expectedJoinTypes.contains(joinType)) {
+            return;
+        }
+
+        JoinInfo joinInfo = joinRel.analyzeCondition();
+        if (joinInfo.leftSet().isEmpty()) {
+            if (joinFilter == null) {
+                return;
+            }
+            // build the new JoinInfo from the join filter if the original JoinInfo has empty keys
+            joinInfo = JoinInfo.of(joinRel.getLeft(), joinRel.getRight(), joinFilter);
+            if (joinInfo.leftSet().isEmpty()) {
+                return;
+            }
+        }
+
+        int leftFieldCnt = joinRel.getLeft().getRowType().getFieldList().size();
+        ImmutableBitSet rightKeyBitsWithOffset =
+                ImmutableBitSet.of(
+                        joinInfo.rightKeys.stream()
+                                .map(i -> i + leftFieldCnt)
+                                .collect(Collectors.toList()));
+
+        for (RexNode filter : filtersToPush) {
+            final RelOptUtil.InputFinder inputFinder = RelOptUtil.InputFinder.analyze(filter);
+            final ImmutableBitSet inputBits = inputFinder.build();
+            if (filter.isAlwaysTrue()) {
+                continue;
+            }
+
+            if (joinInfo.leftSet().contains(inputBits)) {
+                final RexNode shiftedFilter =
+                        remapFilter(
+                                joinInfo.leftKeys,
+                                joinInfo.rightKeys,
+                                joinRel.getRight().getRowType(),
+                                filter);
+                if (!rightFilters.contains(shiftedFilter)) {
+                    rightFilters.add(shiftedFilter);
+                }
+            } else if (rightKeyBitsWithOffset.contains(inputBits)) {
+                ImmutableIntList rightKeysWithOffset =
+                        ImmutableIntList.copyOf(
+                                joinInfo.rightKeys.stream()
+                                        .map(i -> i + leftFieldCnt)
+                                        .collect(Collectors.toList()));
+                final RexNode shiftedFilter =
+                        remapFilter(
+                                rightKeysWithOffset,
+                                joinInfo.leftKeys,
+                                joinRel.getLeft().getRowType(),
+                                filter);
+                if (!leftFilters.contains(shiftedFilter)) {
+                    leftFilters.add(shiftedFilter);
+                }
+            }
+        }
+    }
+
+    private RexNode remapFilter(
+            ImmutableIntList oldKeys,
+            ImmutableIntList newKeys,
+            RelDataType newInputType,
+            RexNode filter) {
+        Map<Integer, Integer> mapping = new HashMap<>();
+        for (int i = 0; i < oldKeys.size(); ++i) {
+            mapping.put(oldKeys.get(i), newKeys.get(i));
+        }
+        RexShuttle shuttle =
+                new RexShuttle() {
+                    @Override
+                    public RexNode visitInputRef(RexInputRef inputRef) {
+                        int newIndex = mapping.getOrDefault(inputRef.getIndex(), -1);
+                        if (newIndex < 0) {
+                            throw new TableException("should not happen");
+                        }
+                        return new RexInputRef(
+                                newIndex, newInputType.getFieldList().get(newIndex).getType());
+                    }
+                };
+        return filter.accept(shuttle);
+    }
+
+    /** Rule that pushes parts of the join condition to its inputs. */
+    public static class FlinkJoinConditionPushRule
+            extends FlinkFilterJoinRule<FlinkJoinConditionPushRule.Config> {
+        /** Creates a JoinConditionPushRule. */
+        protected FlinkJoinConditionPushRule(FlinkJoinConditionPushRule.Config config) {
+            super(config);
+        }
+
+        @Override
+        public void onMatch(RelOptRuleCall call) {
+            Join join = call.rel(0);
+            perform(call, null, join);
+        }
+
+        /** Rule configuration. */
+        public interface Config extends FlinkFilterJoinRule.Config {
+            FlinkJoinConditionPushRule.Config DEFAULT =
+                    EMPTY.withOperandSupplier(b -> b.operand(Join.class).anyInputs())
+                            .as(FlinkJoinConditionPushRule.Config.class)
+                            .withSmart(true)
+                            .withPredicate((join, joinType, exp) -> true)
+                            .as(FlinkJoinConditionPushRule.Config.class);
+
+            @Override
+            default FlinkJoinConditionPushRule toRule() {
+                return new FlinkJoinConditionPushRule(this);
+            }
+        }
+    }
+
+    /**
+     * Rule that tries to push filter expressions into a join condition and into the inputs of the
+     * join.
+     *
+     * @see CoreRules#FILTER_INTO_JOIN
+     */
+    public static class FlinkFilterIntoJoinRule
+            extends FlinkFilterJoinRule<FlinkFilterIntoJoinRule.Config> {
+        /** Creates a FlinkFilterIntoJoinRule. */
+        protected FlinkFilterIntoJoinRule(FlinkFilterIntoJoinRule.Config config) {
+            super(config);
+        }
+
+        @Override
+        public void onMatch(RelOptRuleCall call) {
+            Filter filter = call.rel(0);
+            Join join = call.rel(1);
+            perform(call, filter, join);
+        }
+
+        /** Rule configuration. */
+        public interface Config extends FlinkFilterJoinRule.Config {
+            FlinkFilterIntoJoinRule.Config DEFAULT =
+                    EMPTY.withOperandSupplier(
+                                    b0 ->
+                                            b0.operand(Filter.class)
+                                                    .oneInput(
+                                                            b1 ->
+                                                                    b1.operand(Join.class)
+                                                                            .anyInputs()))
+                            .as(FlinkFilterIntoJoinRule.Config.class)
+                            .withSmart(true)
+                            .withPredicate((join, joinType, exp) -> true)
+                            .as(FlinkFilterIntoJoinRule.Config.class);
+
+            @Override
+            default FlinkFilterIntoJoinRule toRule() {
+                return new FlinkFilterIntoJoinRule(this);
+            }
+        }
+    }
+
+    /**
+     * Predicate that returns whether a filter is valid in the ON clause of a join for this
+     * particular kind of join. If not, Calcite will push it back to above the join.
+     */
+    @FunctionalInterface
+    public interface Predicate {
+        boolean apply(Join join, JoinRelType joinType, RexNode exp);
+    }
+
+    /** Rule configuration. */
+    public interface Config extends RelRule.Config {
+        /** Whether to try to strengthen join-type, default false. */
+        @ImmutableBeans.Property
+        @ImmutableBeans.BooleanDefault(false)
+        boolean isSmart();
+
+        /** Sets {@link #isSmart()}. */
+        Config withSmart(boolean smart);
+
+        /**
+         * Predicate that returns whether a filter is valid in the ON clause of a join for this
+         * particular kind of join. If not, Calcite will push it back to above the join.
+         */
+        @ImmutableBeans.Property
+        Predicate getPredicate();
+
+        /** Sets {@link #getPredicate()} ()}. */
+        Config withPredicate(Predicate predicate);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala
index 6513355e61b..a5b1b197dec 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkBatchRuleSets.scala
@@ -125,9 +125,9 @@ object FlinkBatchRuleSets {
   /** RuleSet about filter */
   private val FILTER_RULES: RuleSet = RuleSets.ofList(
     // push a filter into a join
-    CoreRules.FILTER_INTO_JOIN,
+    FlinkFilterJoinRule.FILTER_INTO_JOIN,
     // push filter into the children of a join
-    CoreRules.JOIN_CONDITION_PUSH,
+    FlinkFilterJoinRule.JOIN_CONDITION_PUSH,
     // push filter through an aggregation
     CoreRules.FILTER_AGGREGATE_TRANSPOSE,
     // push a filter past a project
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
index d7ba45ebf71..d5462682964 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/FlinkStreamRuleSets.scala
@@ -134,9 +134,9 @@ object FlinkStreamRuleSets {
   /** RuleSet about filter */
   private val FILTER_RULES: RuleSet = RuleSets.ofList(
     // push a filter into a join
-    CoreRules.FILTER_INTO_JOIN,
+    FlinkFilterJoinRule.FILTER_INTO_JOIN,
     // push filter into the children of a join
-    CoreRules.JOIN_CONDITION_PUSH,
+    FlinkFilterJoinRule.JOIN_CONDITION_PUSH,
     // push filter through an aggregation
     CoreRules.FILTER_AGGREGATE_TRANSPOSE,
     // push a filter past a project
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/BroadcastJoinHintTest.java
similarity index 94%
rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/BroadcastJoinHintTest.java
index 4a96fcd3bbc..30d7e290fca 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/BroadcastJoinHintTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.planner.plan.batch.sql.join.joinhint;
+package org.apache.flink.table.planner.plan.batch.sql.join.hints;
 
 import org.apache.flink.table.planner.hint.JoinStrategy;
 
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/JoinHintTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/JoinHintTestBase.java
similarity index 99%
rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/JoinHintTestBase.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/JoinHintTestBase.java
index d19f3547516..c3f7473d6ca 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/JoinHintTestBase.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/JoinHintTestBase.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.planner.plan.batch.sql.join.joinhint;
+package org.apache.flink.table.planner.plan.batch.sql.join.hints;
 
 import org.apache.flink.table.api.ExplainDetail;
 import org.apache.flink.table.api.SqlParserException;
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/NestLoopJoinHintTest.java
similarity index 94%
rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/NestLoopJoinHintTest.java
index 5c8b1e8f3e8..40dcb9ba0e3 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/NestLoopJoinHintTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.planner.plan.batch.sql.join.joinhint;
+package org.apache.flink.table.planner.plan.batch.sql.join.hints;
 
 import org.apache.flink.table.planner.hint.JoinStrategy;
 
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleHashJoinHintTest.java
similarity index 94%
rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleHashJoinHintTest.java
index bd19dcc53eb..23bbb51d5d7 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleHashJoinHintTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.planner.plan.batch.sql.join.joinhint;
+package org.apache.flink.table.planner.plan.batch.sql.join.hints;
 
 import org.apache.flink.table.planner.hint.JoinStrategy;
 
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleMergeJoinHintTest.java
similarity index 94%
rename from flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.java
rename to flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleMergeJoinHintTest.java
index 0cb31812184..41a126ddeed 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleMergeJoinHintTest.java
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.planner.plan.batch.sql.join.joinhint;
+package org.apache.flink.table.planner.plan.batch.sql.join.hints;
 
 import org.apache.flink.table.planner.hint.JoinStrategy;
 
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java
index 6b6d1a3fad6..4cbbd62106c 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java
@@ -22,7 +22,7 @@ import org.apache.flink.table.api.StatementSet;
 import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.internal.StatementSetImpl;
 import org.apache.flink.table.planner.hint.JoinStrategy;
-import org.apache.flink.table.planner.plan.batch.sql.join.joinhint.JoinHintTestBase;
+import org.apache.flink.table.planner.plan.batch.sql.join.hints.JoinHintTestBase;
 import org.apache.flink.table.planner.utils.TableTestBase;
 import org.apache.flink.table.planner.utils.TableTestUtil;
 
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java
index bc458317c02..fcc6f87927d 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java
@@ -22,7 +22,7 @@ import org.apache.flink.table.api.StatementSet;
 import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.internal.StatementSetImpl;
 import org.apache.flink.table.planner.hint.JoinStrategy;
-import org.apache.flink.table.planner.plan.batch.sql.join.joinhint.JoinHintTestBase;
+import org.apache.flink.table.planner.plan.batch.sql.join.hints.JoinHintTestBase;
 import org.apache.flink.table.planner.utils.TableTestBase;
 import org.apache.flink.table.planner.utils.TableTestUtil;
 
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.java
new file mode 100644
index 00000000000..c1d195dceef
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.java
@@ -0,0 +1,313 @@
+/*
+ * 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.TableConfig;
+import org.apache.flink.table.planner.calcite.CalciteConfig;
+import org.apache.flink.table.planner.plan.optimize.program.BatchOptimizeContext;
+import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram;
+import org.apache.flink.table.planner.plan.optimize.program.FlinkHepRuleSetProgramBuilder;
+import org.apache.flink.table.planner.plan.optimize.program.HEP_RULES_EXECUTION_TYPE;
+import org.apache.flink.table.planner.utils.BatchTableTestUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
+import org.apache.flink.table.planner.utils.TableTestBase;
+
+import org.apache.calcite.plan.hep.HepMatchOrder;
+import org.apache.calcite.rel.rules.CoreRules;
+import org.apache.calcite.tools.RuleSets;
+import org.junit.Before;
+import org.junit.Test;
+
+/** Test for {@link FlinkFilterJoinRule}. */
+public class FlinkFilterJoinRuleTest extends TableTestBase {
+
+    private BatchTableTestUtil util;
+
+    @Before
+    public void setup() {
+        util = batchTestUtil(TableConfig.getDefault());
+        util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE());
+        CalciteConfig calciteConfig =
+                TableConfigUtils.getCalciteConfig(util.tableEnv().getConfig());
+        calciteConfig
+                .getBatchProgram()
+                .get()
+                .addLast(
+                        "rules",
+                        FlinkHepRuleSetProgramBuilder.<BatchOptimizeContext>newBuilder()
+                                .setHepRulesExecutionType(
+                                        HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION())
+                                .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
+                                .add(
+                                        RuleSets.ofList(
+                                                CoreRules.FILTER_PROJECT_TRANSPOSE,
+                                                FlinkFilterJoinRule.FILTER_INTO_JOIN,
+                                                FlinkFilterJoinRule.JOIN_CONDITION_PUSH))
+                                .build());
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE leftT (\n"
+                                + "  a INT,\n"
+                                + "  b BIGINT\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE rightT (\n"
+                                + "  c INT,\n"
+                                + "  d BIGINT\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE MyTable1 (\n"
+                                + "  a1 INT,\n"
+                                + "  b1 BIGINT,\n"
+                                + "  c1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE MyTable2 (\n"
+                                + "  b2 BIGINT,\n"
+                                + "  c2 VARCHAR,\n"
+                                + "  a2 INT\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+    }
+
+    @Test
+    public void testFilterPushDownLeftSemi1() {
+        util.verifyRelPlan(
+                "SELECT * FROM (SELECT * FROM leftT WHERE a IN (SELECT c FROM rightT)) T WHERE T.b > 2");
+    }
+
+    @Test
+    public void testFilterPushDownLeftSemi2() {
+        util.verifyRelPlan(
+                "SELECT * FROM (SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT)) T WHERE T.b > 2");
+    }
+
+    @Test
+    public void testFilterPushDownLeftSemi3() {
+        util.verifyRelPlan(
+                "SELECT * FROM (SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2");
+    }
+
+    @Test
+    public void testJoinConditionPushDownLeftSemi1() {
+        util.verifyRelPlan("SELECT * FROM leftT WHERE a IN (SELECT c FROM rightT WHERE b > 2)");
+    }
+
+    @Test
+    public void testJoinConditionPushDownLeftSemi2() {
+        util.verifyRelPlan("SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT WHERE b > 2)");
+    }
+
+    @Test
+    public void testJoinConditionPushDownLeftSemi3() {
+        util.verifyRelPlan(
+                "SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT WHERE a = c AND b > 2)");
+    }
+
+    @Test
+    public void testFilterPushDownLeftAnti1() {
+        util.verifyRelPlan(
+                "SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE c < 3)) T WHERE T.b > 2");
+    }
+
+    @Test
+    public void testFilterPushDownLeftAnti2() {
+        util.verifyRelPlan(
+                "SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT where c > 10)) T WHERE T.b > 2");
+    }
+
+    @Test
+    public void testFilterPushDownLeftAnti3() {
+        util.verifyRelPlan(
+                "SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE b = d AND c < 3)) T WHERE T.b > 2");
+    }
+
+    @Test
+    public void testFilterPushDownLeftAnti4() {
+        util.verifyRelPlan(
+                "SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2");
+    }
+
+    @Test
+    public void testJoinConditionPushDownLeftAnti1() {
+        util.verifyRelPlan("SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE b > 2)");
+    }
+
+    @Test
+    public void testJoinConditionPushDownLeftAnti2() {
+        util.verifyRelPlan(
+                "SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT WHERE b > 2)");
+    }
+
+    @Test
+    public void testJoinConditionPushDownLeftAnti3() {
+        util.verifyRelPlan(
+                "SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE b = d AND b > 1)");
+    }
+
+    @Test
+    public void testJoinConditionPushDownLeftAnti4() {
+        util.verifyRelPlan(
+                "SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT WHERE a = c AND b > 2)");
+    }
+
+    @Test
+    public void testInnerJoinWithAllFilterFromBothSide() {
+        // can not be pushed down
+        util.verifyRelPlan("SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 WHERE a1 = a2 + 2");
+    }
+
+    @Test
+    public void testInnerJoinWithAllFilterInONClause() {
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND a1 = 2 AND b2 > 10");
+    }
+
+    @Test
+    public void testInnerJoinWithSomeFiltersFromLeftSide() {
+        util.verifyRelPlan("SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 WHERE a1 = 2");
+    }
+
+    @Test
+    public void testInnerJoinWithSomeFiltersFromRightSide() {
+        util.verifyRelPlan("SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 WHERE a2 = 2");
+    }
+
+    @Test
+    public void testInnerJoinWithSomeFiltersFromLeftRightSide() {
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL");
+    }
+
+    @Test
+    public void testInnerJoinWithAllFiltersFromWhere() {
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable2, MyTable1 WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' ");
+    }
+
+    @Test
+    public void testLeftJoinWithSomeFiltersFromLeftSide() {
+        util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a1 = 2");
+    }
+
+    @Test
+    public void testLeftJoinWithAllFilterInONClause() {
+        util.verifyRelPlan("SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 AND a2 = 2");
+    }
+
+    @Test
+    public void testLeftJoinWithSomeFiltersFromLeftRightSide() {
+        // will be converted to inner join
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL");
+    }
+
+    @Test
+    public void testLeftJoinWithAllFiltersFromWhere() {
+        // will be converted to inner join
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON true WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' ");
+    }
+
+    @Test
+    public void testRightJoinWithAllFilterInONClause() {
+        util.verifyRelPlan("SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON a1 = a2 AND a1 = 2");
+    }
+
+    @Test
+    public void testRightJoinWithSomeFiltersFromRightSide() {
+        util.verifyRelPlan("SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON a1 = a2 WHERE a2 = 2");
+    }
+
+    @Test
+    public void testRightJoinWithSomeFiltersFromLeftRightSide() {
+        // will be converted to inner join
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL");
+    }
+
+    @Test
+    public void testRightJoinWithAllFiltersFromWhere() {
+        // will be converted to inner join
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON true WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' ");
+    }
+
+    @Test
+    public void testFullJoinWithAllFilterInONClause() {
+        util.verifyRelPlan("SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND a1 = 2");
+    }
+
+    @Test
+    public void testFullJoinWithSomeFiltersFromLeftSide() {
+        // will be converted to inner join
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND b1 = b2 WHERE a1 = 2");
+    }
+
+    @Test
+    public void testFullJoinWithSomeFiltersFromRightSide() {
+        // will be converted to inner join
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND b1 = b2 WHERE a2 = 2");
+    }
+
+    @Test
+    public void testFullJoinWithSomeFiltersFromLeftRightSide() {
+        // will be converted to inner join
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL");
+    }
+
+    @Test
+    public void testFullJoinWithAllFiltersFromWhere() {
+        // will be converted to inner join
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable2, MyTable1 WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' ");
+    }
+
+    @Test
+    public void testSemiJoin() {
+        // TODO can not be pushed down now, support it later
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 WHERE (a1, b1, c1) IN (SELECT a2, b2, c2 FROM MyTable2 WHERE a2 = 2 AND b2 > 10) AND c1 IS NOT NULL");
+    }
+
+    @Test
+    public void testAntiJoin() {
+        // can not be pushed down
+        util.verifyRelPlan(
+                "SELECT * FROM MyTable1 WHERE (a1, b1, c1) NOT IN (select a2, b2, c2 FROM MyTable2 WHERE a2 = 2 AND b2 > 10) AND c1 IS NOT NULL");
+    }
+}
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml
index be020d7c557..1e7a5089851 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.xml
@@ -444,19 +444,21 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3], a0=[$4], b0=[$5], d0=[$6], e0=[$7
       <![CDATA[
 HashJoin(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, e0], build=[right])
 :- Exchange(distribution=[hash[b]], shuffle_mode=[BATCH])
-:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left])
+:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[right])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b], where=[(a < 10)])
 :     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1])
-:     +- Exchange(distribution=[hash[d]])(reuse_id=[2])
-:        +- Calc(select=[d, e])
-:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:     +- Exchange(distribution=[hash[d]])
+:        +- Calc(select=[d, e], where=[(d < 10)])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])(reuse_id=[2])
 +- Exchange(distribution=[hash[e]])
-   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left])
+   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[right])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b], where=[(a > 5)])
       :     +- Reused(reference_id=[1])
-      +- Reused(reference_id=[2])
+      +- Exchange(distribution=[hash[d]])
+         +- Calc(select=[d, e], where=[(d > 5)])
+            +- Reused(reference_id=[2])
 ]]>
     </Resource>
   </TestCase>
@@ -492,7 +494,8 @@ Calc(select=[a, b, a0, b0, c, k, CAST(0 AS BIGINT) AS v])
    +- Calc(select=[a, b, c, a0 AS k])
       +- HashJoin(joinType=[InnerJoin], where=[((a = a0) AND ((b + b0) = 0))], select=[a, b, c, a0, b0], build=[right])
          :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])
-         :  +- Reused(reference_id=[1])
+         :  +- Calc(select=[a, b, c], where=[(a > 0)])
+         :     +- Reused(reference_id=[1])
          +- Exchange(distribution=[hash[a]])
             +- Reused(reference_id=[2])
 ]]>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DynamicFilteringTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DynamicFilteringTest.xml
index 75d522ca41a..39f8afbc4ab 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DynamicFilteringTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/DynamicFilteringTest.xml
@@ -30,10 +30,10 @@ HashJoin(joinType=[InnerJoin], where=[=(p1, p)], select=[a1, b1, c1, p1, x, y, z
 :- Exchange(distribution=[hash[p1]])
 :  +- DynamicFilteringTableSourceScan(table=[[default_catalog, default_database, fact1, partitions=[{p1=2}, {p1=3}]]], fields=[a1, b1, c1, p1])
 :     +- DynamicFilteringDataCollector(fields=[p])
-:        +- Calc(select=[x, y, z, p], where=[>(x, 10)])
+:        +- Calc(select=[x, y, z, p], where=[AND(>(x, 10), >(CAST(p AS BIGINT), 1))])
 :           +- TableSourceScan(table=[[default_catalog, default_database, dim, filter=[]]], fields=[x, y, z, p])
 +- Exchange(distribution=[hash[p]])
-   +- Calc(select=[x, y, z, p], where=[>(x, 10)])
+   +- Calc(select=[x, y, z, p], where=[AND(>(x, 10), >(CAST(p AS BIGINT), 1))])
       +- TableSourceScan(table=[[default_catalog, default_database, dim, filter=[]]], fields=[x, y, z, p])
 
 == Optimized Execution Plan ==
@@ -41,7 +41,7 @@ HashJoin(joinType=[InnerJoin], where=[(p1 = p)], select=[a1, b1, c1, p1, x, y, z
 :- Exchange(distribution=[hash[p1]])
 :  +- DynamicFilteringTableSourceScan(table=[[default_catalog, default_database, fact1, partitions=[{p1=2}, {p1=3}]]], fields=[a1, b1, c1, p1])
 :     +- DynamicFilteringDataCollector(fields=[p])
-:        +- Calc(select=[x, y, z, p], where=[(x > 10)])(reuse_id=[1])
+:        +- Calc(select=[x, y, z, p], where=[((x > 10) AND (CAST(p AS BIGINT) > 1))])(reuse_id=[1])
 :           +- TableSourceScan(table=[[default_catalog, default_database, dim, filter=[]]], fields=[x, y, z, p])
 +- Exchange(distribution=[hash[p]])
    +- Reused(reference_id=[1])
@@ -64,7 +64,7 @@ HashJoin(joinType=[InnerJoin], where=[(p1 = p)], select=[a1, b1, c1, p1, x, y, z
     "id" : ,
     "type" : "Calc[]",
     "pact" : "Operator",
-    "contents" : "[]:Calc(select=[x, y, z, p], where=[(x > 10)])",
+    "contents" : "[]:Calc(select=[x, y, z, p], where=[((x > 10) AND (CAST(p AS BIGINT) > 1))])",
     "parallelism" : 1,
     "predecessors" : [ {
       "id" : ,
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml
index e9a55fca5fc..71e534e4058 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.xml
@@ -104,9 +104,11 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4], a0=[$5], b=[$6], c=[$7])
       <![CDATA[
 MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = a0)], select=[a, d, e, f, ny, a0, b, c], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, d, e, f, ny], build=[right])\n:  :- Calc(select=[a], where=[(a > 10)])\n:  :  +- [#3] BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])\n:  +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n])
 :- Exchange(distribution=[broadcast])
-:  +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c])
+:  +- Calc(select=[a, b, c], where=[(a > 10)])
+:     +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c])
 :- Exchange(distribution=[broadcast])
-:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny])
+:  +- Calc(select=[d, e, f, ny], where=[(d > 10)])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny])
 +- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])
 ]]>
     </Resource>
@@ -435,7 +437,8 @@ Calc(select=[a, b, a0, b0, c, nx, k, CAST(0 AS BIGINT) AS v])
    :  +- Calc(select=[a, b], where=[(a > 0)])(reuse_id=[2])
    :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1])
    :- Exchange(distribution=[hash[a]])
-   :  +- Reused(reference_id=[1])
+   :  +- Calc(select=[a, b, c, nx], where=[(a > 0)])
+   :     +- Reused(reference_id=[1])
    +- Exchange(distribution=[hash[a]])
       +- Reused(reference_id=[2])
 ]]>
@@ -471,7 +474,8 @@ Calc(select=[a, b, a0, b0, c, nx, k, CAST(0 AS BIGINT) AS v])
    :  +- Calc(select=[a, b], where=[(a > 0)])(reuse_id=[2])
    :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c, nx)]]], fields=[a, b, c, nx])(reuse_id=[1])
    :- Exchange(distribution=[hash[a]], shuffle_mode=[BATCH])
-   :  +- Reused(reference_id=[1])
+   :  +- Calc(select=[a, b, c, nx], where=[(a > 0)])
+   :     +- Reused(reference_id=[1])
    +- Exchange(distribution=[hash[a]])
       +- Reused(reference_id=[2])
 ]]>
@@ -571,9 +575,11 @@ LogicalProject(a=[$0], d=[$1], e=[$2], f=[$3], ny=[$4], a0=[$5], b=[$6], c=[$7])
       <![CDATA[
 MultipleInput(readOrder=[0,0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = a0)], select=[a, d, e, f, ny, a0, b, c], build=[right])\n:- NestedLoopJoin(joinType=[LeftOuterJoin], where=[(a = d)], select=[a, d, e, f, ny], build=[right])\n:  :- Calc(select=[a], where=[(a > 10)])\n:  :  +- [#3] BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])\n:  +- [#2] Exchange(distribution=[broadcast])\n+- [#1] Exchange(distribution=[broadcast])\n])
 :- Exchange(distribution=[broadcast])
-:  +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c])
+:  +- Calc(select=[a, b, c], where=[(a > 10)])
+:     +- BoundedStreamScan(table=[[default_catalog, default_database, t]], fields=[a, b, c])
 :- Exchange(distribution=[broadcast])
-:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny])
+:  +- Calc(select=[d, e, f, ny], where=[(d > 10)])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f, ny)]]], fields=[d, e, f, ny])
 +- BoundedStreamScan(table=[[default_catalog, default_database, chainable]], fields=[a])
 ]]>
     </Resource>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml
index 87987d9ddb5..aa4cd5a629e 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.xml
@@ -114,19 +114,21 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4], f=[$5], a0=[$6], b0=[$7],
       <![CDATA[
 HashJoin(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, c, d, e, f, a0, b0, c0, d0, e0, f0], build=[right])
 :- Exchange(distribution=[hash[b]])
-:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left])
+:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[right])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b, c], where=[(a < 10)])
 :     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
-:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- Calc(select=[d, e, f], where=[(d < 10)])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[e]])
-   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[left])
+   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], build=[right])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b, c], where=[(a > 5)])
       :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Exchange(distribution=[hash[d]])
-         +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- Calc(select=[d, e, f], where=[(d > 5)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -161,19 +163,21 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3], a0=[$4], b0=[$5], d0=[$6], e0=[$7
       <![CDATA[
 HashJoin(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, e0], build=[right])
 :- Exchange(distribution=[hash[b]])
-:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left])
+:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[right])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b], where=[(a < 10)])
 :     :     +- TableSourceScan(table=[[default_catalog, default_database, newX, filter=[], project=[a, b], metadata=[]]], fields=[a, b])
 :     +- Exchange(distribution=[hash[d]])
-:        +- TableSourceScan(table=[[default_catalog, default_database, newY, project=[d, e], metadata=[]]], fields=[d, e])
+:        +- Calc(select=[d, e], where=[(d < 10)])
+:           +- TableSourceScan(table=[[default_catalog, default_database, newY, filter=[], project=[d, e], metadata=[]]], fields=[d, e])
 +- Exchange(distribution=[hash[e]])
-   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left])
+   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[right])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b], where=[(a > 5)])
       :     +- TableSourceScan(table=[[default_catalog, default_database, newX, filter=[], project=[a, b], metadata=[]]], fields=[a, b])
       +- Exchange(distribution=[hash[d]])
-         +- TableSourceScan(table=[[default_catalog, default_database, newY, project=[d, e], metadata=[]]], fields=[d, e])
+         +- Calc(select=[d, e], where=[(d > 5)])
+            +- TableSourceScan(table=[[default_catalog, default_database, newY, filter=[], project=[d, e], metadata=[]]], fields=[d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -205,19 +209,21 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3], a0=[$4], b0=[$5], d0=[$6], e0=[$7
       <![CDATA[
 HashJoin(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, e0], build=[right])
 :- Exchange(distribution=[hash[b]], shuffle_mode=[BATCH])
-:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left])
+:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[right])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b], where=[(a < 10)])
 :     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1])
-:     +- Exchange(distribution=[hash[d]])(reuse_id=[2])
-:        +- Calc(select=[d, e])
-:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:     +- Exchange(distribution=[hash[d]])
+:        +- Calc(select=[d, e], where=[(d < 10)])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])(reuse_id=[2])
 +- Exchange(distribution=[hash[e]])
-   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left])
+   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[right])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b], where=[(a > 5)])
       :     +- Reused(reference_id=[1])
-      +- Reused(reference_id=[2])
+      +- Exchange(distribution=[hash[d]])
+         +- Calc(select=[d, e], where=[(d > 5)])
+            +- Reused(reference_id=[2])
 ]]>
     </Resource>
   </TestCase>
@@ -252,18 +258,21 @@ LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3], a0=[$4], b0=[$5], d0=[$6], e0=[$7
       <![CDATA[
 HashJoin(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, e0], build=[right])
 :- Exchange(distribution=[hash[b]], shuffle_mode=[BATCH])
-:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left])
+:  +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[right])
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b], where=[(a < 10)])
 :     :     +- TableSourceScan(table=[[default_catalog, default_database, newX, filter=[], project=[a, b], metadata=[]]], fields=[a, b])(reuse_id=[1])
-:     +- Exchange(distribution=[hash[d]])(reuse_id=[2])
-:        +- TableSourceScan(table=[[default_catalog, default_database, newY, project=[d, e], metadata=[]]], fields=[d, e])
+:     +- Exchange(distribution=[hash[d]])
+:        +- Calc(select=[d, e], where=[(d < 10)])
+:           +- TableSourceScan(table=[[default_catalog, default_database, newY, filter=[], project=[d, e], metadata=[]]], fields=[d, e])(reuse_id=[2])
 +- Exchange(distribution=[hash[e]])
-   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[left])
+   +- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], build=[right])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b], where=[(a > 5)])
       :     +- Reused(reference_id=[1])
-      +- Reused(reference_id=[2])
+      +- Exchange(distribution=[hash[d]])
+         +- Calc(select=[d, e], where=[(d > 5)])
+            +- Reused(reference_id=[2])
 ]]>
     </Resource>
   </TestCase>
@@ -334,7 +343,7 @@ LogicalProject(c=[$0], e=[$1], avg_b=[$2], sum_b=[$3], psum=[$4], nsum=[$5], avg
 Calc(select=[c, e, avg_b, sum_b, sum_b0 AS psum, sum_b1 AS nsum, avg_b0 AS avg_b2])
 +- MultipleInput(readOrder=[2,0,1], members=[\nHashJoin(joinType=[InnerJoin], where=[((c = c0) AND (e = e0) AND (rn = $f5))], select=[sum_b, avg_b, rn, c, e, sum_b0, avg_b0, sum_b1, c0, e0, $f5], build=[left])\n:- Calc(select=[sum_b, avg_b, rn, c, e, sum_b0, avg_b0])\n:  +- HashJoin(joinType=[InnerJoin], where=[((c = c0) AND (e = e0) AND (rn = $f5))], select=[sum_b, avg_b, rn, c, e, sum_b0, avg_b0, c0, e0, $f5], build=[left])\n:     :- [#2] Exchange(distribution=[hash[c, e, rn]])\n:      [...]
    :- Exchange(distribution=[hash[c, e, $f5]], shuffle_mode=[BATCH])
-   :  +- Calc(select=[sum_b, c, e, (w1$o0 - 1) AS $f5])
+   :  +- Calc(select=[sum_b, c, e, (w1$o0 - 1) AS $f5], where=[(c <> '')])
    :     +- OverAggregate(partitionBy=[c, e], window#0=[COUNT(sum_b) AS w0$o0, $SUM0(sum_b) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], window#1=[RANK(*) AS w1$o0 RANG BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW], select=[c, e, sum_b, w0$o0, w0$o1, w1$o0])(reuse_id=[1])
    :        +- Sort(orderBy=[c ASC, e ASC])
    :           +- HashAggregate(isMerge=[true], groupBy=[c, e], select=[c, e, Final_SUM(sum$0) AS sum_b])
@@ -352,7 +361,7 @@ Calc(select=[c, e, avg_b, sum_b, sum_b0 AS psum, sum_b1 AS nsum, avg_b0 AS avg_b
    :  +- Calc(select=[sum_b, (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0) AS avg_b, w1$o0 AS rn, c, e], where=[((c <> '') AND ((sum_b - (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0)) > 3))])
    :     +- Reused(reference_id=[1])
    +- Exchange(distribution=[hash[c, e, $f5]], shuffle_mode=[BATCH])
-      +- Calc(select=[sum_b, (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0) AS avg_b, c, e, (w1$o0 + 1) AS $f5])
+      +- Calc(select=[sum_b, (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0) AS avg_b, c, e, (w1$o0 + 1) AS $f5], where=[(c <> '')])
          +- Reused(reference_id=[1])
 ]]>
     </Resource>
@@ -960,7 +969,7 @@ HashJoin(joinType=[InnerJoin], where=[(a = a0)], select=[c, a, b, c0, a0, b0], b
 :           +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1])
 :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[a]])
-   +- Calc(select=[c, a, b], where=[(b < 10)])
+   +- Calc(select=[c, a, b], where=[((b < 10) AND (a > 1))])
       +- Reused(reference_id=[1])
 ]]>
     </Resource>
@@ -1060,8 +1069,8 @@ LogicalProject(c=[$0], a=[$1], b=[$2], c0=[$3], a0=[$4], b0=[$5])
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-NestedLoopJoin(joinType=[InnerJoin], where=[(a = a0)], select=[c, a, b, c0, a0, b0], build=[left])
-:- Exchange(distribution=[broadcast])
+NestedLoopJoin(joinType=[InnerJoin], where=[(a = a0)], select=[c, a, b, c0, a0, b0], build=[right])
+:- Exchange(distribution=[any], shuffle_mode=[BATCH])
 :  +- Calc(select=[c, a, b], where=[(a > 1)])
 :     +- SortLimit(orderBy=[a ASC, b DESC], offset=[0], fetch=[10], global=[true])(reuse_id=[1])
 :        +- Exchange(distribution=[single])
@@ -1070,8 +1079,8 @@ NestedLoopJoin(joinType=[InnerJoin], where=[(a = a0)], select=[c, a, b, c0, a0,
 :                 +- Exchange(distribution=[hash[c]])
 :                    +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(a) AS sum$0, Partial_SUM(b) AS sum$1])
 :                       +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-+- Exchange(distribution=[any], shuffle_mode=[BATCH])
-   +- Calc(select=[c, a, b], where=[(b < 10)])
++- Exchange(distribution=[broadcast])
+   +- Calc(select=[c, a, b], where=[((b < 10) AND (a > 1))])
       +- Reused(reference_id=[1])
 ]]>
     </Resource>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml
index a4a456a0a9d..20d2814d731 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.xml
@@ -31,12 +31,12 @@ LogicalProject(c=[$7], g=[$3])
     <Resource name="optimized exec plan">
       <![CDATA[
 Calc(select=[c, g])
-+- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], isBroadcast=[true], build=[left])
-   :- Exchange(distribution=[broadcast])
-   :  +- Calc(select=[d, g], where=[(d < 2)])
-   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-   +- Calc(select=[a, c])
-      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], isBroadcast=[true], build=[right])
+   :- Calc(select=[d, g], where=[(d < 2)])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   +- Exchange(distribution=[broadcast])
+      +- Calc(select=[a, c], where=[(a < 2)])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -75,11 +75,37 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], isBroadcast=[true], build=[left])
-:- Exchange(distribution=[broadcast])
-:  +- Calc(select=[d, e, f, g, h], where=[(d < 2)])
-:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], isBroadcast=[true], build=[right])
+:- Calc(select=[d, e, f, g, h], where=[(d < 2)])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- Exchange(distribution=[broadcast])
+   +- Calc(select=[a, b, c], where=[(a < 2)])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[SELECT c, g FROM MyTable1, MyTable2 WHERE a = d]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(c=[$2], g=[$6])
++- LogicalFilter(condition=[=($0, $3)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[c, g])
++- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g], isBroadcast=[true], build=[right])
+   :- Calc(select=[a, c])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[broadcast])
+      +- Calc(select=[d, g])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -99,12 +125,52 @@ LogicalProject(c=[$7], g=[$3])
     <Resource name="optimized exec plan">
       <![CDATA[
 Calc(select=[c, g])
-+- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], isBroadcast=[true], build=[left])
++- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], isBroadcast=[true], build=[right])
+   :- Calc(select=[d, g], where=[(d < 2)])
+   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   +- Exchange(distribution=[broadcast])
+      +- Calc(select=[a, c], where=[(a < 2)])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and d = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e])
++- MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e], isBroadcast=[true], build=[right])\n:- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:  +- [#2] Exchange(distribution=[hash[a]])\n+- [#1] Exchange(distribution=[broadcast])\n])
    :- Exchange(distribution=[broadcast])
-   :  +- Calc(select=[d, g], where=[(d < 2)])
-   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-   +- Calc(select=[a, c])
-      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+   :     +- Exchange(distribution=[hash[d]])
+   :        +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   +- Exchange(distribution=[hash[a]])
+      +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -177,6 +243,85 @@ Calc(select=[a, d])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[d])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   left join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and a = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($0, 2))])
+   +- LogicalJoin(condition=[true], joinType=[left])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[CAST(2 AS INTEGER) AS a, b, d, CAST(e AS BIGINT) AS e])
++- MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e], isBroadcast=[true], build=[right])\n:- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:  +- [#2] Exchange(distribution=[hash[a]])\n+- [#1] Exchange(distribution=[broadcast])\n])
+   :- Exchange(distribution=[broadcast])
+   :  +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+   :     +- Exchange(distribution=[hash[d]])
+   :        +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   +- Exchange(distribution=[hash[a]])
+      +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   left join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and a = 2 and e = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (b = 1) AND (a = 2))], select=[a, b, d, e], isBroadcast=[true], build=[right])\n:- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:  +- [#2] Exchange(distribution=[hash[a]])\n+- [#1] Exchange(distribution=[broadcast])\n])
+:- Exchange(distribution=[broadcast])
+:  +- Calc(select=[d, e], where=[(e = 1)])
+:     +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+:        +- Exchange(distribution=[hash[d]])
+:           +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+:              +- Calc(select=[d, e])
+:                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- Exchange(distribution=[hash[a]])
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+      +- Calc(select=[a, b])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -228,27 +373,121 @@ Calc(select=[c, g])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testInnerJoinWithEquiPred">
+  <TestCase name="testInnerJoinWithJoinConditionPushDown">
     <Resource name="sql">
-      <![CDATA[SELECT c, g FROM MyTable1, MyTable2 WHERE a = d]]>
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and d = 2 and b = 1
+]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
-LogicalProject(c=[$2], g=[$6])
-+- LogicalFilter(condition=[=($0, $3)])
-   +- LogicalJoin(condition=[true], joinType=[inner])
-      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
-      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[inner])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
 ]]>
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-Calc(select=[c, g])
-+- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g], isBroadcast=[true], build=[right])
-   :- Calc(select=[a, c])
-   :  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-   +- Exchange(distribution=[broadcast])
-      +- Calc(select=[d, g])
+Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e])
++- MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], isBroadcast=[true], build=[right])\n:- Calc(select=[a], where=[(b = 1)])\n:  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:     +- [#2] Exchange(distribution=[hash[a]])\n+- [#1] Exchange(distribution=[broadcast])\n])
+   :- Exchange(distribution=[broadcast])
+   :  +- Calc(select=[d], where=[(e = 1)])
+   :     +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+   :        +- Exchange(distribution=[hash[d]])
+   :           +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+   :              +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+   :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   +- Exchange(distribution=[hash[a]])
+      +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   right join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and d = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[right])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a, CAST(b AS BIGINT) AS b, CAST(2 AS INTEGER) AS d, e])
++- MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e], isBroadcast=[true], build=[right])\n:- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:  +- [#2] Exchange(distribution=[hash[a]])\n+- [#1] Exchange(distribution=[broadcast])\n])
+   :- Exchange(distribution=[broadcast])
+   :  +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+   :     +- Exchange(distribution=[hash[d]])
+   :        +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+   +- Exchange(distribution=[hash[a]])
+      +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   right join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and d = 2 and b = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+MultipleInput(readOrder=[0,1], members=[\nHashJoin(joinType=[RightOuterJoin], where=[((a = d) AND (1 = e) AND (d = 2))], select=[a, b, d, e], isBroadcast=[true], build=[left])\n:- [#1] Exchange(distribution=[broadcast])\n+- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n   +- [#2] Exchange(distribution=[hash[d]])\n])
+:- Exchange(distribution=[broadcast])
+:  +- Calc(select=[a, b], where=[(b = 1)])
+:     +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+:        +- Exchange(distribution=[hash[a]])
+:           +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+:              +- Calc(select=[a, b])
+:                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- Exchange(distribution=[hash[d]])
+   +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+      +- Calc(select=[d, e])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml
index 292fcdb069c..9c8477cde77 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.xml
@@ -221,7 +221,7 @@ Calc(select=[c, g])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[d, g], where=[(d < 2)])
    :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-   +- Calc(select=[a, c])
+   +- Calc(select=[a, c], where=[(a < 2)])
       +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
@@ -261,11 +261,37 @@ LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-NestedLoopJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[left])
-:- Exchange(distribution=[broadcast])
-:  +- Calc(select=[d, e, f, g, h], where=[(d < 2)])
-:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+NestedLoopJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[right])
+:- Calc(select=[d, e, f, g, h], where=[(d < 2)])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- Exchange(distribution=[broadcast])
+   +- Calc(select=[a, b, c], where=[(a < 2)])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[SELECT c, g FROM MyTable1, MyTable2 WHERE a = d]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(c=[$2], g=[$6])
++- LogicalFilter(condition=[=($0, $3)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[c, g])
++- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- Calc(select=[a, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Calc(select=[d, g])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -289,8 +315,48 @@ Calc(select=[c, g])
    :- Exchange(distribution=[broadcast])
    :  +- Calc(select=[d, g], where=[(d < 2)])
    :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-   +- Calc(select=[a, c])
+   +- Calc(select=[a, c], where=[(a < 2)])
       +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and d = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e])
++- MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e], build=[left])\n:- [#1] Exchange(distribution=[broadcast])\n+- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n   +- [#2] Exchange(distribution=[hash[d]])\n])
+   :- Exchange(distribution=[broadcast])
+   :  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :     +- Exchange(distribution=[hash[a]])
+   :        +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d]])
+      +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -386,6 +452,85 @@ NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a, d], build=[left])
 :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Calc(select=[d])
    +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   left join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and a = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($0, 2))])
+   +- LogicalJoin(condition=[true], joinType=[left])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[CAST(2 AS INTEGER) AS a, b, d, CAST(e AS BIGINT) AS e])
++- MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e], build=[left])\n:- [#1] Exchange(distribution=[broadcast])\n+- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n   +- [#2] Exchange(distribution=[hash[d]])\n])
+   :- Exchange(distribution=[broadcast])
+   :  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :     +- Exchange(distribution=[hash[a]])
+   :        +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d]])
+      +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   left join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and a = 2 and e = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (b = 1) AND (a = 2))], select=[a, b, d, e], build=[right])\n:- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:  +- [#2] Exchange(distribution=[hash[a]])\n+- [#1] Exchange(distribution=[broadcast])\n])
+:- Exchange(distribution=[broadcast])
+:  +- Calc(select=[d, e], where=[(e = 1)])
+:     +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+:        +- Exchange(distribution=[hash[d]])
+:           +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+:              +- Calc(select=[d, e])
+:                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- Exchange(distribution=[hash[a]])
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+      +- Calc(select=[a, b])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -500,49 +645,43 @@ Calc(select=[c, g])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testInnerJoinWithEquiPred">
+  <TestCase name="testInnerJoinWithJoinConditionPushDown">
     <Resource name="sql">
-      <![CDATA[SELECT c, g FROM MyTable1, MyTable2 WHERE a = d]]>
-    </Resource>
-    <Resource name="ast">
       <![CDATA[
-LogicalProject(c=[$2], g=[$6])
-+- LogicalFilter(condition=[=($0, $3)])
-   +- LogicalJoin(condition=[true], joinType=[inner])
-      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
-      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and d = 2 and b = 1
 ]]>
     </Resource>
-    <Resource name="optimized exec plan">
-      <![CDATA[
-Calc(select=[c, g])
-+- NestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g], build=[left])
-   :- Exchange(distribution=[broadcast])
-   :  +- Calc(select=[a, c])
-   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-   +- Calc(select=[d, g])
-      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-]]>
-    </Resource>
-  </TestCase>
-  <TestCase name="testRightOuterJoinOnFalse">
-    <Resource name="sql">
-      <![CDATA[SELECT * FROM MyTable2 RIGHT OUTER JOIN MyTable1 ON false]]>
-    </Resource>
     <Resource name="ast">
       <![CDATA[
-LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
-+- LogicalJoin(condition=[false], joinType=[right])
-   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[inner])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
 ]]>
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-NestedLoopJoin(joinType=[RightOuterJoin], where=[true], select=[d, e, f, g, h, a, b, c], build=[left])
-:- Exchange(distribution=[broadcast])
-:  +- Values(tuples=[[]], values=[d, e, f, g, h])
-+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e])
++- MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[left])\n:- [#1] Exchange(distribution=[broadcast])\n+- Calc(select=[d], where=[(e = 1)])\n   +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n      +- [#2] Exchange(distribution=[hash[d]])\n])
+   :- Exchange(distribution=[broadcast])
+   :  +- Calc(select=[a], where=[(b = 1)])
+   :     +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :        +- Exchange(distribution=[hash[a]])
+   :           +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :              +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d]])
+      +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -567,6 +706,106 @@ Calc(select=[c, g])
    +- Exchange(distribution=[broadcast])
       +- Calc(select=[e, g])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   right join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and d = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[right])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a, CAST(b AS BIGINT) AS b, CAST(2 AS INTEGER) AS d, e])
++- MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e], build=[left])\n:- [#1] Exchange(distribution=[broadcast])\n+- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n   +- [#2] Exchange(distribution=[hash[d]])\n])
+   :- Exchange(distribution=[broadcast])
+   :  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :     +- Exchange(distribution=[hash[a]])
+   :        +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d]])
+      +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   right join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and d = 2 and b = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+MultipleInput(readOrder=[0,1], members=[\nNestedLoopJoin(joinType=[RightOuterJoin], where=[((a = d) AND (1 = e) AND (d = 2))], select=[a, b, d, e], build=[left])\n:- [#1] Exchange(distribution=[broadcast])\n+- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n   +- [#2] Exchange(distribution=[hash[d]])\n])
+:- Exchange(distribution=[broadcast])
+:  +- Calc(select=[a, b], where=[(b = 1)])
+:     +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+:        +- Exchange(distribution=[hash[a]])
+:           +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+:              +- Calc(select=[a, b])
+:                 +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- Exchange(distribution=[hash[d]])
+   +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+      +- Calc(select=[d, e])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightOuterJoinOnFalse">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable2 RIGHT OUTER JOIN MyTable1 ON false]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
++- LogicalJoin(condition=[false], joinType=[right])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[RightOuterJoin], where=[true], select=[d, e, f, g, h, a, b, c], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- Values(tuples=[[]], values=[d, e, f, g, h])
++- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml
index 8fcc68cdde5..8b50a7ce09f 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml
@@ -130,58 +130,63 @@ LogicalProject(c=[$7], g=[$3])
     <Resource name="optimized exec plan">
       <![CDATA[
 Calc(select=[c, g])
-+- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], build=[left])
++- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], build=[right])
    :- Exchange(distribution=[hash[d]])
    :  +- Calc(select=[d, g], where=[(d < 2)])
    :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Exchange(distribution=[hash[a]])
-      +- Calc(select=[a, c])
+      +- Calc(select=[a, c], where=[(a < 2)])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testLeftOuterJoinWithEquiAndNonEquiPred">
+  <TestCase name="testInnerJoinWithEquiAndNonEquiPred">
     <Resource name="sql">
-      <![CDATA[SELECT * FROM MyTable2 LEFT OUTER JOIN  MyTable1 ON a = d AND d < 2 AND b < h]]>
+      <![CDATA[SELECT * FROM MyTable2 INNER JOIN MyTable1 ON a = d AND d < 2 AND b < h]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
 LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
-+- LogicalJoin(condition=[AND(=($5, $0), <($0, 2), <($6, $4))], joinType=[left])
++- LogicalJoin(condition=[AND(=($5, $0), <($0, 2), <($6, $4))], joinType=[inner])
    :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
    +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
 ]]>
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-HashJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[right])
+HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[right])
 :- Exchange(distribution=[hash[d]])
-:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+:  +- Calc(select=[d, e, f, g, h], where=[(d < 2)])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[hash[a]])
-   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Calc(select=[a, b, c], where=[(a < 2)])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testInnerJoinWithEquiAndNonEquiPred">
+  <TestCase name="testInnerJoinWithEquiPred">
     <Resource name="sql">
-      <![CDATA[SELECT * FROM MyTable2 INNER JOIN MyTable1 ON a = d AND d < 2 AND b < h]]>
+      <![CDATA[SELECT c, g FROM MyTable1, MyTable2 WHERE a = d]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
-LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
-+- LogicalJoin(condition=[AND(=($5, $0), <($0, 2), <($6, $4))], joinType=[inner])
-   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+LogicalProject(c=[$2], g=[$6])
++- LogicalFilter(condition=[=($0, $3)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
 ]]>
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[left])
-:- Exchange(distribution=[hash[d]])
-:  +- Calc(select=[d, e, f, g, h], where=[(d < 2)])
-:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- Exchange(distribution=[hash[a]])
-   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+Calc(select=[c, g])
++- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g], build=[right])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d]])
+      +- Calc(select=[d, g])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -201,13 +206,55 @@ LogicalProject(c=[$7], g=[$3])
     <Resource name="optimized exec plan">
       <![CDATA[
 Calc(select=[c, g])
-+- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], build=[left])
++- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[d, g, a, c], build=[right])
    :- Exchange(distribution=[hash[d]])
    :  +- Calc(select=[d, g], where=[(d < 2)])
    :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Exchange(distribution=[hash[a]])
-      +- Calc(select=[a, c])
+      +- Calc(select=[a, c], where=[(a < 2)])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and d = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e])
++- HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e], build=[right])
+   :- Exchange(distribution=[hash[a, b]])
+   :  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :     +- Exchange(distribution=[hash[a]])
+   :        +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d, e]])
+      +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+         +- Exchange(distribution=[hash[d]])
+            +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+               +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -233,6 +280,43 @@ Calc(select=[c, g])
    +- Exchange(distribution=[hash[e, d]])
       +- Calc(select=[d, e, g])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and d = 2 and b = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[inner])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e])
++- MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d], build=[right])\n:- Calc(select=[a], where=[(b = 1)])\n:  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:     +- [#1] Exchange(distribution=[hash[a]])\n+- Calc(select=[d], where=[(e = 1)])\n   +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n      +- [#2] Exchange(distribution=[hash[d]])\n])
+   :- Exchange(distribution=[hash[a]])
+   :  +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :     +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d]])
+      +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -283,6 +367,84 @@ Calc(select=[a, d])
    +- Exchange(distribution=[hash[d]])
       +- Calc(select=[d])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   left join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and a = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($0, 2))])
+   +- LogicalJoin(condition=[true], joinType=[left])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[CAST(2 AS INTEGER) AS a, b, d, CAST(e AS BIGINT) AS e])
++- HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e], build=[right])
+   :- Exchange(distribution=[hash[a, b]])
+   :  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :     +- Exchange(distribution=[hash[a]])
+   :        +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d, e]])
+      +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+         +- Exchange(distribution=[hash[d]])
+            +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+               +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   left join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and a = 2 and e = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (b = 1) AND (a = 2))], select=[a, b, d, e], build=[right])\n:- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:  +- [#1] Exchange(distribution=[hash[a]])\n+- Calc(select=[d, e], where=[(e = 1)])\n   +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n      +- [#2] Exchange(distribution=[hash[d]])\n])
+:- Exchange(distribution=[hash[a]])
+:  +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+:     +- Calc(select=[a, b])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- Exchange(distribution=[hash[d]])
+   +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+      +- Calc(select=[d, e])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -311,54 +473,57 @@ Calc(select=[c, g])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testRightOuterJoinWithEquiPred">
+  <TestCase name="testSelfJoin">
     <Resource name="sql">
-      <![CDATA[SELECT c, g FROM MyTable1 RIGHT OUTER JOIN MyTable2 ON b = e]]>
+      <![CDATA[SELECT * FROM
+  (SELECT * FROM src WHERE k = 0) src1
+LEFT OUTER JOIN
+  (SELECT * from src WHERE k = 0) src2
+ON (src1.k = src2.k AND src2.k > 10)
+         ]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
-LogicalProject(c=[$2], g=[$6])
-+- LogicalJoin(condition=[=($1, $4)], joinType=[right])
-   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+LogicalProject(k=[$0], v=[$1], k0=[$2], v0=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), >($2, 10))], joinType=[left])
+   :- LogicalProject(k=[$0], v=[$1])
+   :  +- LogicalFilter(condition=[=($0, 0)])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]])
+   +- LogicalProject(k=[$0], v=[$1])
+      +- LogicalFilter(condition=[=($0, 0)])
+         +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]])
 ]]>
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-Calc(select=[c, g])
-+- HashJoin(joinType=[RightOuterJoin], where=[(b = e)], select=[b, c, e, g], build=[right])
-   :- Exchange(distribution=[hash[b]])
-   :  +- Calc(select=[b, c])
-   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-   +- Exchange(distribution=[hash[e]])
-      +- Calc(select=[e, g])
-         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+HashJoin(joinType=[LeftOuterJoin], where=[(k = k0)], select=[k, v, k0, v0], build=[right])
+:- Exchange(distribution=[hash[k]])
+:  +- Calc(select=[CAST(0 AS BIGINT) AS k, v], where=[(k = 0)])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]], fields=[k, v])
++- Exchange(distribution=[hash[k]])
+   +- Values(tuples=[[]], values=[k, v])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testInnerJoinWithEquiPred">
+  <TestCase name="testLeftOuterJoinWithEquiAndNonEquiPred">
     <Resource name="sql">
-      <![CDATA[SELECT c, g FROM MyTable1, MyTable2 WHERE a = d]]>
+      <![CDATA[SELECT * FROM MyTable2 LEFT OUTER JOIN  MyTable1 ON a = d AND d < 2 AND b < h]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
-LogicalProject(c=[$2], g=[$6])
-+- LogicalFilter(condition=[=($0, $3)])
-   +- LogicalJoin(condition=[true], joinType=[inner])
-      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
-      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
++- LogicalJoin(condition=[AND(=($5, $0), <($0, 2), <($6, $4))], joinType=[left])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
 ]]>
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-Calc(select=[c, g])
-+- HashJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g], build=[right])
-   :- Exchange(distribution=[hash[a]])
-   :  +- Calc(select=[a, c])
-   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-   +- Exchange(distribution=[hash[d]])
-      +- Calc(select=[d, g])
-         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+HashJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (d < 2) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[right])
+:- Exchange(distribution=[hash[d]])
+:  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- Exchange(distribution=[hash[a]])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
@@ -387,26 +552,45 @@ Calc(select=[c, g])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testRightOuterJoinWithEquiAndNonEquiPred">
+  <TestCase name="testRightJoinWithFilterPushDown">
     <Resource name="sql">
-      <![CDATA[SELECT * FROM MyTable2 RIGHT OUTER JOIN  MyTable1 ON a = d AND d < 2 AND b < h]]>
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   right join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and d = 2
+]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
-LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
-+- LogicalJoin(condition=[AND(=($5, $0), <($0, 2), <($6, $4))], joinType=[right])
-   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[right])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
 ]]>
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-HashJoin(joinType=[RightOuterJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[left])
-:- Exchange(distribution=[hash[d]])
-:  +- Calc(select=[d, e, f, g, h], where=[(d < 2)])
-:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
-+- Exchange(distribution=[hash[a]])
-   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+Calc(select=[a, CAST(b AS BIGINT) AS b, CAST(2 AS INTEGER) AS d, e])
++- HashJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e], build=[right])
+   :- Exchange(distribution=[hash[a, b]])
+   :  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :     +- Exchange(distribution=[hash[a]])
+   :        +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d, e]])
+      +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+         +- Exchange(distribution=[hash[d]])
+            +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+               +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -435,35 +619,87 @@ Calc(select=[c, g])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testSelfJoin">
+  <TestCase name="testRightOuterJoinWithEquiAndNonEquiPred">
     <Resource name="sql">
-      <![CDATA[SELECT * FROM
-  (SELECT * FROM src WHERE k = 0) src1
-LEFT OUTER JOIN
-  (SELECT * from src WHERE k = 0) src2
-ON (src1.k = src2.k AND src2.k > 10)
-         ]]>
+      <![CDATA[SELECT * FROM MyTable2 RIGHT OUTER JOIN  MyTable1 ON a = d AND d < 2 AND b < h]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
-LogicalProject(k=[$0], v=[$1], k0=[$2], v0=[$3])
-+- LogicalJoin(condition=[AND(=($0, $2), >($2, 10))], joinType=[left])
-   :- LogicalProject(k=[$0], v=[$1])
-   :  +- LogicalFilter(condition=[=($0, 0)])
-   :     +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]])
-   +- LogicalProject(k=[$0], v=[$1])
-      +- LogicalFilter(condition=[=($0, 0)])
-         +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]])
+LogicalProject(d=[$0], e=[$1], f=[$2], g=[$3], h=[$4], a=[$5], b=[$6], c=[$7])
++- LogicalJoin(condition=[AND(=($5, $0), <($0, 2), <($6, $4))], joinType=[right])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
 ]]>
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-HashJoin(joinType=[LeftOuterJoin], where=[(k = k0)], select=[k, v, k0, v0], build=[right])
-:- Exchange(distribution=[hash[k]])
-:  +- Calc(select=[CAST(0 AS BIGINT) AS k, v], where=[(k = 0)])
-:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]], fields=[k, v])
-+- Exchange(distribution=[hash[k]])
-   +- Values(tuples=[[]], values=[k, v])
+HashJoin(joinType=[RightOuterJoin], where=[((a = d) AND (b < h))], select=[d, e, f, g, h, a, b, c], build=[left])
+:- Exchange(distribution=[hash[d]])
+:  +- Calc(select=[d, e, f, g, h], where=[(d < 2)])
+:     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- Exchange(distribution=[hash[a]])
+   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightOuterJoinWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[SELECT c, g FROM MyTable1 RIGHT OUTER JOIN MyTable2 ON b = e]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(c=[$2], g=[$6])
++- LogicalJoin(condition=[=($1, $4)], joinType=[right])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[c, g])
++- HashJoin(joinType=[RightOuterJoin], where=[(b = e)], select=[b, c, e, g], build=[right])
+   :- Exchange(distribution=[hash[b]])
+   :  +- Calc(select=[b, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[e]])
+      +- Calc(select=[e, g])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   right join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and d = 2 and b = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+MultipleInput(readOrder=[1,0], members=[\nHashJoin(joinType=[RightOuterJoin], where=[((a = d) AND (1 = e) AND (d = 2))], select=[a, b, d, e], build=[left])\n:- Calc(select=[a, b], where=[(b = 1)])\n:  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:     +- [#2] Exchange(distribution=[hash[a]])\n+- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n   +- [#1] Exchange(distribution=[hash[d]])\n])
+:- Exchange(distribution=[hash[d]])
+:  +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+:     +- Calc(select=[d, e])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- Exchange(distribution=[hash[a]])
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+      +- Calc(select=[a, b])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml
index 921f5e0cd2f..836d0f99a90 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.xml
@@ -135,7 +135,7 @@ Calc(select=[c, g])
    :  +- Calc(select=[d, g], where=[(d < 2)])
    :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Exchange(distribution=[hash[a]])
-      +- Calc(select=[a, c])
+      +- Calc(select=[a, c], where=[(a < 2)])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
@@ -181,7 +181,34 @@ SortMergeJoin(joinType=[InnerJoin], where=[((a = d) AND (b < h))], select=[d, e,
 :  +- Calc(select=[d, e, f, g, h], where=[(d < 2)])
 :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 +- Exchange(distribution=[hash[a]])
-   +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Calc(select=[a, b, c], where=[(a < 2)])
+      +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[SELECT c, g FROM MyTable1, MyTable2 WHERE a = d]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(c=[$2], g=[$6])
++- LogicalFilter(condition=[=($0, $3)])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[c, g])
++- SortMergeJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, c])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d]])
+      +- Calc(select=[d, g])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -206,8 +233,50 @@ Calc(select=[c, g])
    :  +- Calc(select=[d, g], where=[(d < 2)])
    :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
    +- Exchange(distribution=[hash[a]])
-      +- Calc(select=[a, c])
+      +- Calc(select=[a, c], where=[(a < 2)])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and d = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a, b, CAST(2 AS INTEGER) AS d, e])
++- SortMergeJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e])
+   :- Exchange(distribution=[hash[a, b]])
+   :  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :     +- Exchange(distribution=[hash[a]])
+   :        +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d, e]])
+      +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+         +- Exchange(distribution=[hash[d]])
+            +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+               +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -283,6 +352,84 @@ Calc(select=[a, d])
    +- Exchange(distribution=[hash[d]])
       +- Calc(select=[d])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   left join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and a = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($0, 2))])
+   +- LogicalJoin(condition=[true], joinType=[left])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[CAST(2 AS INTEGER) AS a, b, d, CAST(e AS BIGINT) AS e])
++- SortMergeJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e])
+   :- Exchange(distribution=[hash[a, b]])
+   :  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :     +- Exchange(distribution=[hash[a]])
+   :        +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d, e]])
+      +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+         +- Exchange(distribution=[hash[d]])
+            +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+               +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   left join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and a = 2 and e = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+MultipleInput(members=[\nSortMergeJoin(joinType=[LeftOuterJoin], where=[((a = d) AND (b = 1) AND (a = 2))], select=[a, b, d, e])\n:- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:  +- [#1] Exchange(distribution=[hash[a]])\n+- Calc(select=[d, e], where=[(e = 1)])\n   +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n      +- [#2] Exchange(distribution=[hash[d]])\n])
+:- Exchange(distribution=[hash[a]])
+:  +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+:     +- Calc(select=[a, b])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
++- Exchange(distribution=[hash[d]])
+   +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+      +- Calc(select=[d, e])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
 ]]>
     </Resource>
   </TestCase>
@@ -336,29 +483,118 @@ Calc(select=[c, g])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testInnerJoinWithEquiPred">
+  <TestCase name="testInnerJoinWithJoinConditionPushDown">
     <Resource name="sql">
-      <![CDATA[SELECT c, g FROM MyTable1, MyTable2 WHERE a = d]]>
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and d = 2 and b = 1
+]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
-LogicalProject(c=[$2], g=[$6])
-+- LogicalFilter(condition=[=($0, $3)])
-   +- LogicalJoin(condition=[true], joinType=[inner])
-      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
-      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[inner])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
 ]]>
     </Resource>
     <Resource name="optimized exec plan">
       <![CDATA[
-Calc(select=[c, g])
-+- SortMergeJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, c, d, g])
+Calc(select=[a, 1 AS b, CAST(2 AS INTEGER) AS d, 1 AS e])
++- MultipleInput(members=[\nSortMergeJoin(joinType=[InnerJoin], where=[(a = d)], select=[a, d])\n:- Calc(select=[a], where=[(b = 1)])\n:  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:     +- [#1] Exchange(distribution=[hash[a]])\n+- Calc(select=[d], where=[(e = 1)])\n   +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n      +- [#2] Exchange(distribution=[hash[d]])\n])
    :- Exchange(distribution=[hash[a]])
-   :  +- Calc(select=[a, c])
-   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   :  +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :     +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
    +- Exchange(distribution=[hash[d]])
-      +- Calc(select=[d, g])
-         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+      +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+         +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   right join
+   (select d, count(e) as e from MyTable2 group by d)
+   on true where a = d and b = e and d = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[right])
+      :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+      :  +- LogicalProject(a=[$0], b=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+      +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+         +- LogicalProject(d=[$0], e=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a, CAST(b AS BIGINT) AS b, CAST(2 AS INTEGER) AS d, e])
++- SortMergeJoin(joinType=[InnerJoin], where=[((a = d) AND (b = e))], select=[a, b, d, e])
+   :- Exchange(distribution=[hash[a, b]])
+   :  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])
+   :     +- Exchange(distribution=[hash[a]])
+   :        +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a, b], where=[(a = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[d, e]])
+      +- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])
+         +- Exchange(distribution=[hash[d]])
+            +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+               +- Calc(select=[CAST(2 AS INTEGER) AS d, e], where=[(d = 2)])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a, count(b) as b from MyTable1 group by a)
+   right join
+   (select d, count(e) as e from MyTable2 group by d)
+   on a = d and b = e and d = 2 and b = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a=[$0], b=[$1], d=[$2], e=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right])
+   :- LogicalAggregate(group=[{0}], b=[COUNT($1)])
+   :  +- LogicalProject(a=[$0], b=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]])
+   +- LogicalAggregate(group=[{0}], e=[COUNT($1)])
+      +- LogicalProject(d=[$0], e=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+MultipleInput(members=[\nSortMergeJoin(joinType=[RightOuterJoin], where=[((a = d) AND (1 = e) AND (d = 2))], select=[a, b, d, e])\n:- Calc(select=[a, b], where=[(b = 1)])\n:  +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_COUNT(count$0) AS b])\n:     +- [#2] Exchange(distribution=[hash[a]])\n+- HashAggregate(isMerge=[true], groupBy=[d], select=[d, Final_COUNT(count$0) AS e])\n   +- [#1] Exchange(distribution=[hash[d]])\n])
+:- Exchange(distribution=[hash[d]])
+:  +- LocalHashAggregate(groupBy=[d], select=[d, Partial_COUNT(e) AS count$0])
+:     +- Calc(select=[d, e])
+:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable2, source: [TestTableSource(d, e, f, g, h)]]], fields=[d, e, f, g, h])
++- Exchange(distribution=[hash[a]])
+   +- LocalHashAggregate(groupBy=[a], select=[a, Partial_COUNT(b) AS count$0])
+      +- Calc(select=[a, b])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/BroadcastJoinHintTest.xml
similarity index 97%
rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.xml
rename to flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/BroadcastJoinHintTest.xml
index dc60bb7e784..9b0a7acf579 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/BroadcastJoinHintTest.xml
@@ -110,26 +110,30 @@ LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[right])
-      :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :  +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
       +- Exchange(distribution=[broadcast])
          +- Calc(select=[a1])
             +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
                :- Exchange(distribution=[broadcast])
                :  +- Calc(select=[a1], where=[>(a1, 2)])
                :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1])
-               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+               +- Calc(select=[a2], where=[>(a2, 2)])
+                  +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[], project=[a2], metadata=[]]], fields=[a2])
 
 Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left])
       :- Exchange(distribution=[broadcast])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
             :- Exchange(distribution=[broadcast])
             :  +- Calc(select=[a1], where=[>(a1, 5)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1])
-            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+            +- Calc(select=[a2], where=[>(a2, 5)])
+               +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[], project=[a2], metadata=[]]], fields=[a2])
 ]]>
     </Resource>
   </TestCase>
@@ -162,28 +166,30 @@ LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[right])
-      :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Exchange(distribution=[broadcast])
          +- Calc(select=[a1])
             +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
                :- Exchange(distribution=[broadcast])
                :  +- Calc(select=[a1], where=[>(a1, 2)])
                :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
-               +- Calc(select=[a2])
+               +- Calc(select=[a2], where=[>(a2, 2)])
                   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
 
 Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
             :- Exchange(distribution=[hash[a1]])
             :  +- Calc(select=[a1], where=[>(a1, 5)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
             +- Exchange(distribution=[hash[a2]])
-               +- Calc(select=[a2])
+               +- Calc(select=[a2], where=[>(a2, 5)])
                   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
 ]]>
     </Resource>
@@ -217,7 +223,8 @@ LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[right])
-      :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Exchange(distribution=[broadcast])
          +- Calc(select=[a1], where=[>(a1, 2)])
             +- Calc(select=[a1, b1])
@@ -230,7 +237,8 @@ Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left])
       :- Exchange(distribution=[broadcast])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Calc(select=[a1], where=[>(a1, 5)])
          +- Calc(select=[a1, b1])
             +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], isBroadcast=[true], build=[left])
@@ -302,7 +310,8 @@ HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBro
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
-+- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Calc(select=[a2, b2], where=[>(a2, 5)])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
@@ -468,7 +477,8 @@ HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBro
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[a1, b1], where=[<(a1, 1)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
-+- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Calc(select=[a2, b2], where=[<(a2, 1)])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
@@ -535,7 +545,8 @@ HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBro
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[a1, b1], where=[>(a1, 1)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
-+- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Calc(select=[a2, b2], where=[>(a2, 1)])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/NestLoopJoinHintTest.xml
similarity index 97%
rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.xml
rename to flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/NestLoopJoinHintTest.xml
index 3dc576a0029..1fc76daeff7 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/NestLoopJoinHintTest.xml
@@ -110,26 +110,30 @@ LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[right])
-      :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :  +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
       +- Exchange(distribution=[broadcast])
          +- Calc(select=[a1])
             +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
                :- Exchange(distribution=[broadcast])
                :  +- Calc(select=[a1], where=[>(a1, 2)])
                :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1])
-               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+               +- Calc(select=[a2], where=[>(a2, 2)])
+                  +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[], project=[a2], metadata=[]]], fields=[a2])
 
 Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
       :- Exchange(distribution=[broadcast])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
             :- Exchange(distribution=[broadcast])
             :  +- Calc(select=[a1], where=[>(a1, 5)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1])
-            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+            +- Calc(select=[a2], where=[>(a2, 5)])
+               +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[], project=[a2], metadata=[]]], fields=[a2])
 ]]>
     </Resource>
   </TestCase>
@@ -162,27 +166,29 @@ LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[right])
-      :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Exchange(distribution=[broadcast])
          +- Calc(select=[a1])
             +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
                :- Exchange(distribution=[broadcast])
                :  +- Calc(select=[a1], where=[>(a1, 2)])
                :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
-               +- Calc(select=[a2])
+               +- Calc(select=[a2], where=[>(a2, 2)])
                   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
 
 Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left])
       :- Exchange(distribution=[broadcast])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
             :- Exchange(distribution=[broadcast])
             :  +- Calc(select=[a1], where=[>(a1, 5)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
-            +- Calc(select=[a2])
+            +- Calc(select=[a2], where=[>(a2, 5)])
                +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
 ]]>
     </Resource>
@@ -216,7 +222,8 @@ LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[right])
-      :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Exchange(distribution=[broadcast])
          +- Calc(select=[a1], where=[>(a1, 2)])
             +- Calc(select=[a1, b1])
@@ -229,7 +236,8 @@ Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
       :- Exchange(distribution=[broadcast])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Calc(select=[a1], where=[>(a1, 5)])
          +- Calc(select=[a1, b1])
             +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], build=[left])
@@ -301,7 +309,8 @@ NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2],
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
-+- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Calc(select=[a2, b2], where=[>(a2, 5)])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
@@ -467,7 +476,8 @@ NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2],
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[a1, b1], where=[<(a1, 1)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
-+- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Calc(select=[a2, b2], where=[<(a2, 1)])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
@@ -534,7 +544,8 @@ NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2],
 :- Exchange(distribution=[broadcast])
 :  +- Calc(select=[a1, b1], where=[>(a1, 1)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
-+- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Calc(select=[a2, b2], where=[>(a2, 1)])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleHashJoinHintTest.xml
similarity index 97%
rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.xml
rename to flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleHashJoinHintTest.xml
index 67f9ad612d4..b695c77a555 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleHashJoinHintTest.xml
@@ -113,27 +113,31 @@ Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[right])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
             :- Exchange(distribution=[hash[a1]])
             :  +- Calc(select=[a1], where=[>(a1, 2)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1])
             +- Exchange(distribution=[hash[a2]])
-               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+               +- Calc(select=[a2], where=[>(a2, 2)])
+                  +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[], project=[a2], metadata=[]]], fields=[a2])
 
 Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
             :- Exchange(distribution=[hash[a1]])
             :  +- Calc(select=[a1], where=[>(a1, 5)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1])
             +- Exchange(distribution=[hash[a2]])
-               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+               +- Calc(select=[a2], where=[>(a2, 5)])
+                  +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[], project=[a2], metadata=[]]], fields=[a2])
 ]]>
     </Resource>
   </TestCase>
@@ -167,27 +171,29 @@ Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[right])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
             :- Exchange(distribution=[hash[a1]])
             :  +- Calc(select=[a1], where=[>(a1, 2)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
             +- Exchange(distribution=[hash[a2]])
-               +- Calc(select=[a2])
+               +- Calc(select=[a2], where=[>(a2, 2)])
                   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
 
 Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left])
       :- Exchange(distribution=[broadcast])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
             :- Exchange(distribution=[broadcast])
             :  +- Calc(select=[a1], where=[>(a1, 5)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
-            +- Calc(select=[a2])
+            +- Calc(select=[a2], where=[>(a2, 5)])
                +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
 ]]>
     </Resource>
@@ -222,7 +228,8 @@ Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[right])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Exchange(distribution=[hash[a1]])
          +- Calc(select=[a1], where=[>(a1, 2)])
             +- Calc(select=[a1, b1])
@@ -236,7 +243,8 @@ Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Exchange(distribution=[hash[a1]])
          +- Calc(select=[a1], where=[>(a1, 5)])
             +- Calc(select=[a1, b1])
@@ -313,7 +321,8 @@ HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build
 :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
 +- Exchange(distribution=[hash[a2]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+   +- Calc(select=[a2, b2], where=[>(a2, 5)])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
@@ -484,7 +493,8 @@ HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build
 :  +- Calc(select=[a1, b1], where=[<(a1, 1)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
 +- Exchange(distribution=[hash[a2]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+   +- Calc(select=[a2, b2], where=[<(a2, 1)])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
@@ -554,7 +564,8 @@ HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build
 :  +- Calc(select=[a1, b1], where=[>(a1, 1)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
 +- Exchange(distribution=[hash[a2]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+   +- Calc(select=[a2, b2], where=[>(a2, 1)])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleMergeJoinHintTest.xml
similarity index 97%
rename from flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.xml
rename to flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleMergeJoinHintTest.xml
index 874d00d39ea..d921282af68 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/hints/ShuffleMergeJoinHintTest.xml
@@ -113,27 +113,31 @@ Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2])
             :- Exchange(distribution=[hash[a1]])
             :  +- Calc(select=[a1], where=[>(a1, 2)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1])
             +- Exchange(distribution=[hash[a2]])
-               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+               +- Calc(select=[a2], where=[>(a2, 2)])
+                  +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[], project=[a2], metadata=[]]], fields=[a2])
 
 Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2])
             :- Exchange(distribution=[hash[a1]])
             :  +- Calc(select=[a1], where=[>(a1, 5)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1])
             +- Exchange(distribution=[hash[a2]])
-               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+               +- Calc(select=[a2], where=[>(a2, 5)])
+                  +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[], project=[a2], metadata=[]]], fields=[a2])
 ]]>
     </Resource>
   </TestCase>
@@ -167,27 +171,29 @@ Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2])
             :- Exchange(distribution=[hash[a1]])
             :  +- Calc(select=[a1], where=[>(a1, 2)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
             +- Exchange(distribution=[hash[a2]])
-               +- Calc(select=[a2])
+               +- Calc(select=[a2], where=[>(a2, 2)])
                   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
 
 Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left])
       :- Exchange(distribution=[broadcast])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Calc(select=[a1])
          +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
             :- Exchange(distribution=[broadcast])
             :  +- Calc(select=[a1], where=[>(a1, 5)])
             :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
-            +- Calc(select=[a2])
+            +- Calc(select=[a2], where=[>(a2, 5)])
                +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
 ]]>
     </Resource>
@@ -222,7 +228,8 @@ Sink(table=[default_catalog.default_database.S1], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 2)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Exchange(distribution=[hash[a1]])
          +- Calc(select=[a1], where=[>(a1, 2)])
             +- Calc(select=[a1, b1])
@@ -236,7 +243,8 @@ Sink(table=[default_catalog.default_database.S2], fields=[a1, b1])
 +- Calc(select=[a1, b1])
    +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10])
       :- Exchange(distribution=[hash[a1]])
-      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+      :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
+      :     +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
       +- Exchange(distribution=[hash[a1]])
          +- Calc(select=[a1], where=[>(a1, 5)])
             +- Calc(select=[a1, b1])
@@ -313,7 +321,8 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
 :  +- Calc(select=[a1, b1], where=[>(a1, 5)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
 +- Exchange(distribution=[hash[a2]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+   +- Calc(select=[a2, b2], where=[>(a2, 5)])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
@@ -484,7 +493,8 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
 :  +- Calc(select=[a1, b1], where=[<(a1, 1)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
 +- Exchange(distribution=[hash[a2]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+   +- Calc(select=[a2, b2], where=[<(a2, 1)])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
@@ -554,7 +564,8 @@ SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
 :  +- Calc(select=[a1, b1], where=[>(a1, 1)])
 :     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
 +- Exchange(distribution=[hash[a2]])
-   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+   +- Calc(select=[a2, b2], where=[>(a2, 1)])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, filter=[]]], fields=[a2, b2])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.xml
index 86a1069172b..3a311ca1d60 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.xml
@@ -16,12 +16,36 @@ See the License for the specific language governing permissions and
 limitations under the License.
 -->
 <Root>
-  <TestCase name="testFilterPushDownLeftAnti1">
+  <TestCase name="testAntiJoin">
     <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 WHERE (a1, b1, c1) NOT IN (select a2, b2, c2 FROM MyTable2 WHERE a2 = 2 AND b2 > 10) AND c1 IS NOT NULL]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2])
++- LogicalFilter(condition=[AND(NOT(IN($0, $1, $2, {
+LogicalProject(a2=[$2], b2=[$0], c2=[$1])
+  LogicalFilter(condition=[AND(=($2, 2), >($0, 10))])
+    LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+})), IS NOT NULL($2))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
       <![CDATA[
-SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN
-    (SELECT c FROM rightT WHERE c < 3)) T WHERE T.b > 2
-      ]]>
+LogicalProject(a1=[$0], b1=[$1], c1=[$2])
++- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($3), =($0, $3)), OR(IS NULL($1), IS NULL($4), =($1, $4)), OR(IS NULL($2), IS NULL($5), =($2, $5)))], joinType=[anti])
+   :- LogicalFilter(condition=[IS NOT NULL($2)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalProject(a2=[$2], b2=[$0], c2=[$1])
+      +- LogicalFilter(condition=[AND(=($2, 2), >($0, 10))])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testFilterPushDownLeftAnti1">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE c < 3)) T WHERE T.b > 2]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
@@ -31,9 +55,9 @@ LogicalProject(a=[$0], b=[$1])
       +- LogicalFilter(condition=[NOT(IN($0, {
 LogicalProject(c=[$0])
   LogicalFilter(condition=[<($0, 3)])
-    LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+    LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 }))])
-         +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -42,19 +66,16 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalProject(a=[$0], b=[$1])
    +- LogicalJoin(condition=[OR(IS NULL($0), IS NULL($2), =($0, $2))], joinType=[anti])
       :- LogicalFilter(condition=[>($1, 2)])
-      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
       +- LogicalProject(c=[$0])
          +- LogicalFilter(condition=[<($0, 3)])
-            +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+            +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
   <TestCase name="testFilterPushDownLeftAnti2">
     <Resource name="sql">
-      <![CDATA[
-SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS
-    (SELECT * FROM rightT where c > 10)) T WHERE T.b > 2
-      ]]>
+      <![CDATA[SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT where c > 10)) T WHERE T.b > 2]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
@@ -63,9 +84,9 @@ LogicalProject(a=[$0], b=[$1])
    +- LogicalProject(a=[$0], b=[$1])
       +- LogicalFilter(condition=[NOT(EXISTS({
 LogicalFilter(condition=[>($0, 10)])
-  LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+  LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 }))])
-         +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -74,21 +95,18 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalProject(a=[$0], b=[$1])
    +- LogicalJoin(condition=[$2], joinType=[anti])
       :- LogicalFilter(condition=[>($1, 2)])
-      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
       +- LogicalProject($f0=[IS NOT NULL($0)])
          +- LogicalAggregate(group=[{}], m=[MIN($0)])
             +- LogicalProject(i=[true])
                +- LogicalFilter(condition=[>($0, 10)])
-                  +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+                  +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
   <TestCase name="testFilterPushDownLeftAnti3">
     <Resource name="sql">
-      <![CDATA[
-SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN
-(SELECT c FROM rightT WHERE b = d AND c < 3)) T WHERE T.b > 2
-      ]]>
+      <![CDATA[SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE b = d AND c < 3)) T WHERE T.b > 2]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
@@ -98,9 +116,9 @@ LogicalProject(a=[$0], b=[$1])
       +- LogicalFilter(condition=[NOT(IN($0, {
 LogicalProject(c=[$0])
   LogicalFilter(condition=[AND(=($cor0.b, $1), <($0, 3))])
-    LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+    LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 }))], variablesSet=[[$cor0]])
-         +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -109,19 +127,16 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalProject(a=[$0], b=[$1])
    +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($2), =($0, $2)), =($1, $3))], joinType=[anti])
       :- LogicalFilter(condition=[>($1, 2)])
-      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
       +- LogicalProject(c=[$0], d=[$1])
          +- LogicalFilter(condition=[<($0, 3)])
-            +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+            +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
   <TestCase name="testFilterPushDownLeftAnti4">
     <Resource name="sql">
-      <![CDATA[
-SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS
-    (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2
-      ]]>
+      <![CDATA[SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
@@ -130,9 +145,9 @@ LogicalProject(a=[$0], b=[$1])
    +- LogicalProject(a=[$0], b=[$1])
       +- LogicalFilter(condition=[NOT(EXISTS({
 LogicalFilter(condition=[=($cor0.a, $0)])
-  LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+  LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 }))], variablesSet=[[$cor0]])
-         +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -141,10 +156,10 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalProject(a=[$0], b=[$1])
    +- LogicalJoin(condition=[=($0, $2)], joinType=[anti])
       :- LogicalFilter(condition=[>($1, 2)])
-      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
       +- LogicalProject(c=[$0])
          +- LogicalFilter(condition=[true])
-            +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+            +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
@@ -159,9 +174,9 @@ LogicalProject(a=[$0], b=[$1])
    +- LogicalProject(a=[$0], b=[$1])
       +- LogicalFilter(condition=[IN($0, {
 LogicalProject(c=[$0])
-  LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+  LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 })])
-         +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -170,9 +185,9 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalProject(a=[$0], b=[$1])
    +- LogicalJoin(condition=[=($0, $2)], joinType=[semi])
       :- LogicalFilter(condition=[>($1, 2)])
-      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
       +- LogicalProject(c=[$0])
-         +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
@@ -186,9 +201,9 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalFilter(condition=[>($1, 2)])
    +- LogicalProject(a=[$0], b=[$1])
       +- LogicalFilter(condition=[EXISTS({
-LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 })])
-         +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -197,20 +212,17 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalProject(a=[$0], b=[$1])
    +- LogicalJoin(condition=[$2], joinType=[semi])
       :- LogicalFilter(condition=[>($1, 2)])
-      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
       +- LogicalProject($f0=[IS NOT NULL($0)])
          +- LogicalAggregate(group=[{}], m=[MIN($0)])
             +- LogicalProject(i=[true])
-               +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
   <TestCase name="testFilterPushDownLeftSemi3">
     <Resource name="sql">
-      <![CDATA[
-SELECT * FROM (SELECT * FROM leftT WHERE EXISTS
-    (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2
-      ]]>
+      <![CDATA[SELECT * FROM (SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
@@ -219,9 +231,9 @@ LogicalProject(a=[$0], b=[$1])
    +- LogicalProject(a=[$0], b=[$1])
       +- LogicalFilter(condition=[EXISTS({
 LogicalFilter(condition=[=($cor0.a, $0)])
-  LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+  LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 })], variablesSet=[[$cor0]])
-         +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -230,10 +242,268 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalProject(a=[$0], b=[$1])
    +- LogicalJoin(condition=[=($0, $2)], joinType=[semi])
       :- LogicalFilter(condition=[>($1, 2)])
-      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
       +- LogicalProject(c=[$0])
          +- LogicalFilter(condition=[true])
-            +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+            +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testFullJoinWithSomeFiltersFromRightSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND b1 = b2 WHERE a2 = 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[=($5, 2)])
+   +- LogicalJoin(condition=[AND(=($0, $5), =($1, $3))], joinType=[full])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($1, $3))], joinType=[right])
+   :- LogicalFilter(condition=[=($0, 2)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[=($2, 2)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testFullJoinWithAllFilterInONClause">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND a1 = 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($0, 2))], joinType=[full])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($0, 2))], joinType=[full])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testFullJoinWithAllFiltersFromWhere">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable2, MyTable1 WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(b2=[$0], c2=[$1], a2=[$2], a1=[$3], b1=[$4], c1=[$5])
++- LogicalFilter(condition=[AND(=($4, $0), =($5, $1), =($2, 2), >($0, 10), <>(COALESCE($5, $1), _UTF-16LE''))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(b2=[$0], c2=[$1], a2=[$2], a1=[$3], b1=[$4], c1=[$5])
++- LogicalJoin(condition=[AND(=($4, $0), =($5, $1), <>(COALESCE($5, $1), _UTF-16LE''))], joinType=[inner])
+   :- LogicalFilter(condition=[AND(=($2, 2), >($0, 10))])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+   +- LogicalFilter(condition=[>($1, 10)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testFullJoinWithSomeFiltersFromLeftRightSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[AND(=($5, 2), >($3, 10), IS NOT NULL($2))])
+   +- LogicalJoin(condition=[AND(=($0, $5), =($1, $3), =($2, $4))], joinType=[full])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($1, $3), =($2, $4))], joinType=[inner])
+   :- LogicalFilter(condition=[AND(=($0, 2), >($1, 10), IS NOT NULL($2))])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[AND(=($2, 2), >($0, 10), IS NOT NULL($1))])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testFullJoinWithSomeFiltersFromLeftSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 FULL JOIN MyTable2 ON a1 = a2 AND b1 = b2 WHERE a1 = 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[=($0, 2)])
+   +- LogicalJoin(condition=[AND(=($0, $5), =($1, $3))], joinType=[full])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($1, $3))], joinType=[left])
+   :- LogicalFilter(condition=[=($0, 2)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[=($2, 2)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithAllFilterFromBothSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 WHERE a1 = a2 + 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[=($0, +($5, 2))])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($0, +($5, 2)))], joinType=[inner])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithAllFilterInONClause">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND a1 = 2 AND b2 > 10]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($1, $3), =($0, 2), >($3, 10))], joinType=[inner])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($1, $3))], joinType=[inner])
+   :- LogicalFilter(condition=[AND(=($0, 2), >($1, 10))])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[AND(>($0, 10), =($2, 2))])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithAllFiltersFromWhere">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable2, MyTable1 WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(b2=[$0], c2=[$1], a2=[$2], a1=[$3], b1=[$4], c1=[$5])
++- LogicalFilter(condition=[AND(=($4, $0), =($5, $1), =($2, 2), >($0, 10), <>(COALESCE($5, $1), _UTF-16LE''))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(b2=[$0], c2=[$1], a2=[$2], a1=[$3], b1=[$4], c1=[$5])
++- LogicalJoin(condition=[AND(=($4, $0), =($5, $1), <>(COALESCE($5, $1), _UTF-16LE''))], joinType=[inner])
+   :- LogicalFilter(condition=[AND(=($2, 2), >($0, 10))])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+   +- LogicalFilter(condition=[>($1, 10)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithSomeFiltersFromLeftRightSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[AND(=($5, 2), >($3, 10), IS NOT NULL($2))])
+   +- LogicalJoin(condition=[AND(=($0, $5), =($1, $3), =($2, $4))], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($1, $3), =($2, $4))], joinType=[inner])
+   :- LogicalFilter(condition=[AND(=($0, 2), >($1, 10), IS NOT NULL($2))])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[AND(=($2, 2), >($0, 10), IS NOT NULL($1))])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithSomeFiltersFromLeftSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 WHERE a1 = 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[=($0, 2)])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[=($0, $5)], joinType=[inner])
+   :- LogicalFilter(condition=[=($0, 2)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[=($2, 2)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithSomeFiltersFromRightSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 JOIN MyTable2 ON a1 = a2 WHERE a2 = 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[=($5, 2)])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[inner])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[=($0, $5)], joinType=[inner])
+   :- LogicalFilter(condition=[=($0, 2)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[=($2, 2)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
 ]]>
     </Resource>
   </TestCase>
@@ -247,19 +517,19 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalFilter(condition=[NOT(IN($0, {
 LogicalProject(c=[$0])
   LogicalFilter(condition=[>($cor0.b, 2)])
-    LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+    LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 }))], variablesSet=[[$cor0]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
       <![CDATA[
 LogicalProject(a=[$0], b=[$1])
 +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($2), =($0, $2)), >($1, 2))], joinType=[anti])
-   :- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
    +- LogicalProject(c=[$0])
       +- LogicalFilter(condition=[true])
-         +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
@@ -272,9 +542,9 @@ LogicalProject(a=[$0], b=[$1])
 LogicalProject(a=[$0], b=[$1])
 +- LogicalFilter(condition=[NOT(EXISTS({
 LogicalFilter(condition=[>($cor0.b, 2)])
-  LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+  LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 }))], variablesSet=[[$cor0]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -283,10 +553,10 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalProject(a=[$0], b=[$1])
    +- LogicalJoin(condition=[$2], joinType=[anti])
       :- LogicalProject(a=[$0], b=[$1], $f2=[>($1, 2)])
-      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
       +- LogicalProject
          +- LogicalFilter(condition=[true])
-            +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+            +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
@@ -300,19 +570,19 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalFilter(condition=[NOT(IN($0, {
 LogicalProject(c=[$0])
   LogicalFilter(condition=[AND(=($cor0.b, $1), >($cor0.b, 1))])
-    LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+    LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 }))], variablesSet=[[$cor0]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
       <![CDATA[
 LogicalProject(a=[$0], b=[$1])
 +- LogicalJoin(condition=[AND(OR(IS NULL($0), IS NULL($2), =($0, $2)), =($1, $3), >($1, 1))], joinType=[anti])
-   :- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
    +- LogicalProject(c=[$0], d=[$1])
       +- LogicalFilter(condition=[true])
-         +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
@@ -325,19 +595,19 @@ LogicalProject(a=[$0], b=[$1])
 LogicalProject(a=[$0], b=[$1])
 +- LogicalFilter(condition=[NOT(EXISTS({
 LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.b, 2))])
-  LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+  LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 }))], variablesSet=[[$cor0]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
       <![CDATA[
 LogicalProject(a=[$0], b=[$1])
 +- LogicalJoin(condition=[AND(=($0, $2), >($1, 2))], joinType=[anti])
-   :- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
    +- LogicalProject(c=[$0])
       +- LogicalFilter(condition=[true])
-         +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
@@ -351,9 +621,9 @@ LogicalProject(a=[$0], b=[$1])
 +- LogicalFilter(condition=[IN($0, {
 LogicalProject(c=[$0])
   LogicalFilter(condition=[>($cor0.b, 2)])
-    LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+    LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 })], variablesSet=[[$cor0]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -361,10 +631,10 @@ LogicalProject(c=[$0])
 LogicalProject(a=[$0], b=[$1])
 +- LogicalJoin(condition=[=($0, $2)], joinType=[semi])
    :- LogicalFilter(condition=[>($1, 2)])
-   :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
    +- LogicalProject(c=[$0])
       +- LogicalFilter(condition=[true])
-         +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
@@ -377,9 +647,9 @@ LogicalProject(a=[$0], b=[$1])
 LogicalProject(a=[$0], b=[$1])
 +- LogicalFilter(condition=[EXISTS({
 LogicalFilter(condition=[>($cor0.b, 2)])
-  LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+  LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 })], variablesSet=[[$cor0]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -389,10 +659,10 @@ LogicalProject(a=[$0], b=[$1])
    +- LogicalJoin(condition=[true], joinType=[semi])
       :- LogicalProject(a=[$0], b=[$1], $f2=[>($1, 2)])
       :  +- LogicalFilter(condition=[>($1, 2)])
-      :     +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
       +- LogicalProject
          +- LogicalFilter(condition=[true])
-            +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+            +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 ]]>
     </Resource>
   </TestCase>
@@ -405,9 +675,9 @@ LogicalProject(a=[$0], b=[$1])
 LogicalProject(a=[$0], b=[$1])
 +- LogicalFilter(condition=[EXISTS({
 LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.b, 2))])
-  LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+  LogicalTableScan(table=[[default_catalog, default_database, rightT]])
 })], variablesSet=[[$cor0]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
 ]]>
     </Resource>
     <Resource name="optimized rel plan">
@@ -415,10 +685,225 @@ LogicalFilter(condition=[AND(=($cor0.a, $0), >($cor0.b, 2))])
 LogicalProject(a=[$0], b=[$1])
 +- LogicalJoin(condition=[=($0, $2)], joinType=[semi])
    :- LogicalFilter(condition=[>($1, 2)])
-   :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT, source: [TestTableSource(a, b)]]])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, leftT]])
    +- LogicalProject(c=[$0])
       +- LogicalFilter(condition=[true])
-         +- LogicalTableScan(table=[[default_catalog, default_database, rightT, source: [TestTableSource(c, d)]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, rightT]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithAllFilterInONClause">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 AND a2 = 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($5, 2))], joinType=[left])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[=($0, $5)], joinType=[left])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[=($2, 2)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithAllFilterInONClause">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON a1 = a2 AND a1 = 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($0, 2))], joinType=[right])
+   :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[=($0, $5)], joinType=[right])
+   :- LogicalFilter(condition=[=($0, 2)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithAllFiltersFromWhere">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON true WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[AND(=($1, $3), =($2, $4), =($5, 2), >($3, 10), <>(COALESCE($2, $4), _UTF-16LE''))])
+   +- LogicalJoin(condition=[true], joinType=[left])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($1, $3), =($2, $4), <>(COALESCE($2, $4), _UTF-16LE''))], joinType=[inner])
+   :- LogicalFilter(condition=[>($1, 10)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[AND(=($2, 2), >($0, 10))])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithSomeFiltersFromLeftRightSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[AND(=($5, 2), >($3, 10), IS NOT NULL($2))])
+   +- LogicalJoin(condition=[AND(=($0, $5), =($1, $3), =($2, $4))], joinType=[left])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($1, $3), =($2, $4))], joinType=[inner])
+   :- LogicalFilter(condition=[AND(=($0, 2), >($1, 10), IS NOT NULL($2))])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[AND(=($2, 2), >($0, 10), IS NOT NULL($1))])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithAllFiltersFromWhere">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON true WHERE b1 = b2 AND c1 = c2 AND a2 = 2 AND b2 > 10 AND COALESCE(c1, c2) <> '' ]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[AND(=($1, $3), =($2, $4), =($5, 2), >($3, 10), <>(COALESCE($2, $4), _UTF-16LE''))])
+   +- LogicalJoin(condition=[true], joinType=[right])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($1, $3), =($2, $4), <>(COALESCE($2, $4), _UTF-16LE''))], joinType=[inner])
+   :- LogicalFilter(condition=[>($1, 10)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[AND(=($2, 2), >($0, 10))])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithSomeFiltersFromLeftSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 LEFT JOIN MyTable2 ON a1 = a2 WHERE a1 = 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[=($0, 2)])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[left])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[=($0, $5)], joinType=[left])
+   :- LogicalFilter(condition=[=($0, 2)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[=($2, 2)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithSomeFiltersFromRightSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON a1 = a2 WHERE a2 = 2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[=($5, 2)])
+   +- LogicalJoin(condition=[=($0, $5)], joinType=[right])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[=($0, $5)], joinType=[right])
+   :- LogicalFilter(condition=[=($0, 2)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[=($2, 2)])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithSomeFiltersFromLeftRightSide">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 RIGHT JOIN MyTable2 ON a1 = a2 AND b1 = b2 AND c1 = c2 WHERE a2 = 2 AND b2 > 10 AND c1 IS NOT NULL]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalFilter(condition=[AND(=($5, 2), >($3, 10), IS NOT NULL($2))])
+   +- LogicalJoin(condition=[AND(=($0, $5), =($1, $3), =($2, $4))], joinType=[right])
+      :- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2], b2=[$3], c2=[$4], a2=[$5])
++- LogicalJoin(condition=[AND(=($0, $5), =($1, $3), =($2, $4))], joinType=[inner])
+   :- LogicalFilter(condition=[AND(=($0, 2), >($1, 10), IS NOT NULL($2))])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalFilter(condition=[AND(=($2, 2), >($0, 10), IS NOT NULL($1))])
+      +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSemiJoin">
+    <Resource name="sql">
+      <![CDATA[SELECT * FROM MyTable1 WHERE (a1, b1, c1) IN (SELECT a2, b2, c2 FROM MyTable2 WHERE a2 = 2 AND b2 > 10) AND c1 IS NOT NULL]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2])
++- LogicalFilter(condition=[AND(IN($0, $1, $2, {
+LogicalProject(a2=[$2], b2=[$0], c2=[$1])
+  LogicalFilter(condition=[AND(=($2, 2), >($0, 10))])
+    LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
+}), IS NOT NULL($2))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], c1=[$2])
++- LogicalJoin(condition=[AND(=($0, $3), =($1, $4), =($2, $5))], joinType=[semi])
+   :- LogicalFilter(condition=[IS NOT NULL($2)])
+   :  +- LogicalTableScan(table=[[default_catalog, default_database, MyTable1]])
+   +- LogicalProject(a2=[$2], b2=[$0], c2=[$1])
+      +- LogicalFilter(condition=[AND(=($2, 2), >($0, 10))])
+         +- LogicalTableScan(table=[[default_catalog, default_database, MyTable2]])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.xml
index 8692db5a473..83ce7c4c507 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.xml
@@ -77,14 +77,16 @@ Join(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, c, d, e, f, a0, b0, c
 :     :  +- Calc(select=[a, b, c], where=[(a < 10)])
 :     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 :     +- Exchange(distribution=[hash[d]])
-:        +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:        +- Calc(select=[d, e, f], where=[(d < 10)])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 +- Exchange(distribution=[hash[e]])
    +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, c, d, e, f], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b, c], where=[(a > 5)])
       :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
       +- Exchange(distribution=[hash[d]])
-         +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+         +- Calc(select=[d, e, f], where=[(d > 5)])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
 ]]>
     </Resource>
   </TestCase>
@@ -124,14 +126,16 @@ Join(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, e0]
 :     :  +- Calc(select=[a, b], where=[(a < 10)])
 :     :     +- TableSourceScan(table=[[default_catalog, default_database, newX, filter=[], project=[a, b], metadata=[]]], fields=[a, b])
 :     +- Exchange(distribution=[hash[d]])
-:        +- TableSourceScan(table=[[default_catalog, default_database, newY, project=[d, e], metadata=[]]], fields=[d, e])
+:        +- Calc(select=[d, e], where=[(d < 10)])
+:           +- TableSourceScan(table=[[default_catalog, default_database, newY, filter=[], project=[d, e], metadata=[]]], fields=[d, e])
 +- Exchange(distribution=[hash[e]])
    +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b], where=[(a > 5)])
       :     +- TableSourceScan(table=[[default_catalog, default_database, newX, filter=[], project=[a, b], metadata=[]]], fields=[a, b])
       +- Exchange(distribution=[hash[d]])
-         +- TableSourceScan(table=[[default_catalog, default_database, newY, project=[d, e], metadata=[]]], fields=[d, e])
+         +- Calc(select=[d, e], where=[(d > 5)])
+            +- TableSourceScan(table=[[default_catalog, default_database, newY, filter=[], project=[d, e], metadata=[]]], fields=[d, e])
 ]]>
     </Resource>
   </TestCase>
@@ -167,15 +171,17 @@ Join(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, e0]
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b], where=[(a < 10)])
 :     :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])(reuse_id=[1])
-:     +- Exchange(distribution=[hash[d]])(reuse_id=[2])
-:        +- Calc(select=[d, e])
-:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])
+:     +- Exchange(distribution=[hash[d]])
+:        +- Calc(select=[d, e], where=[(d < 10)])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, y, source: [TestTableSource(d, e, f)]]], fields=[d, e, f])(reuse_id=[2])
 +- Exchange(distribution=[hash[e]])
    +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b], where=[(a > 5)])
       :     +- Reused(reference_id=[1])
-      +- Reused(reference_id=[2])
+      +- Exchange(distribution=[hash[d]])
+         +- Calc(select=[d, e], where=[(d > 5)])
+            +- Reused(reference_id=[2])
 ]]>
     </Resource>
   </TestCase>
@@ -320,14 +326,17 @@ Join(joinType=[InnerJoin], where=[(b = e0)], select=[a, b, d, e, a0, b0, d0, e0]
 :     :- Exchange(distribution=[hash[a]])
 :     :  +- Calc(select=[a, b], where=[(a < 10)])
 :     :     +- TableSourceScan(table=[[default_catalog, default_database, newX, filter=[], project=[a, b], metadata=[]]], fields=[a, b])(reuse_id=[1])
-:     +- Exchange(distribution=[hash[d]])(reuse_id=[2])
-:        +- TableSourceScan(table=[[default_catalog, default_database, newY, project=[d, e], metadata=[]]], fields=[d, e])
+:     +- Exchange(distribution=[hash[d]])
+:        +- Calc(select=[d, e], where=[(d < 10)])
+:           +- TableSourceScan(table=[[default_catalog, default_database, newY, filter=[], project=[d, e], metadata=[]]], fields=[d, e])(reuse_id=[2])
 +- Exchange(distribution=[hash[e]])
    +- Join(joinType=[InnerJoin], where=[(a = d)], select=[a, b, d, e], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
       :- Exchange(distribution=[hash[a]])
       :  +- Calc(select=[a, b], where=[(a > 5)])
       :     +- Reused(reference_id=[1])
-      +- Reused(reference_id=[2])
+      +- Exchange(distribution=[hash[d]])
+         +- Calc(select=[d, e], where=[(d > 5)])
+            +- Reused(reference_id=[2])
 ]]>
     </Resource>
   </TestCase>
@@ -694,7 +703,7 @@ Join(joinType=[InnerJoin], where=[(a = a0)], select=[c, a, b, c0, a0, b0], leftI
 :        +- Exchange(distribution=[hash[c]])
 :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, x, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
 +- Exchange(distribution=[hash[a]])
-   +- Calc(select=[c, a, b], where=[(b < 10)])
+   +- Calc(select=[c, a, b], where=[((b < 10) AND (a > 1))])
       +- Reused(reference_id=[1])
 ]]>
     </Resource>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml
index ede29276be2..1d1fc29ebf8 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.xml
@@ -399,6 +399,88 @@ Join(joinType=[InnerJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpec=[Jo
       +- Exchange(distribution=[hash[b1]], changelogMode=[I])
          +- Calc(select=[b1, b2], changelogMode=[I])
             +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+   (select a1, count(a2) as a2 from A group by a1)
+   join
+   (select b1, count(b2) as b2 from B group by b1)
+   on true where a1 = b1 and a2 = b2 and b1 = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalAggregate(group=[{0}], a2=[COUNT($1)])
+      :  +- LogicalProject(a1=[$0], a2=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]])
+      +- LogicalAggregate(group=[{0}], b2=[COUNT($1)])
+         +- LogicalProject(b1=[$0], b2=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a1, a2, CAST(2 AS INTEGER) AS b1, b2])
++- Join(joinType=[InnerJoin], where=[((a1 = b1) AND (a2 = b2))], select=[a1, a2, b1, b2], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey])
+   :- Exchange(distribution=[hash[a1, a2]])
+   :  +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2])
+   :     +- Exchange(distribution=[hash[a1]])
+   :        +- Calc(select=[CAST(2 AS INTEGER) AS a1, a2], where=[(a1 = 2)])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3])
+   +- Exchange(distribution=[hash[b1, b2]])
+      +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2])
+         +- Exchange(distribution=[hash[b1]])
+            +- Calc(select=[CAST(2 AS INTEGER) AS b1, b2], where=[(b1 = 2)])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testInnerJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+  (select a1, count(a2) as a2 from A group by a1)
+   join
+  (select b1, count(b2) as b2 from B group by b1)
+   on true where a1 = b1 and a2 = b2 and b1 = 2 and a2 = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))])
+   +- LogicalJoin(condition=[true], joinType=[inner])
+      :- LogicalAggregate(group=[{0}], a2=[COUNT($1)])
+      :  +- LogicalProject(a1=[$0], a2=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]])
+      +- LogicalAggregate(group=[{0}], b2=[COUNT($1)])
+         +- LogicalProject(b1=[$0], b2=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a1, 1 AS a2, CAST(2 AS INTEGER) AS b1, 1 AS b2])
++- Join(joinType=[InnerJoin], where=[(a1 = b1)], select=[a1, b1], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1], where=[(a2 = 1)])
+   :     +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2])
+   :        +- Exchange(distribution=[hash[a1]])
+   :           +- Calc(select=[CAST(2 AS INTEGER) AS a1, a2], where=[(a1 = 2)])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3])
+   +- Exchange(distribution=[hash[b1]])
+      +- Calc(select=[b1], where=[(b2 = 1)])
+         +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2])
+            +- Exchange(distribution=[hash[b1]])
+               +- Calc(select=[CAST(2 AS INTEGER) AS b1, b2], where=[(b1 = 2)])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3])
 ]]>
     </Resource>
   </TestCase>
@@ -786,6 +868,85 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D])
             +- Exchange(distribution=[hash[b1]], changelogMode=[I])
                +- Calc(select=[b1, b2], changelogMode=[I])
                   +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+  (select a1, count(a2) as a2 from A group by a1)
+   left join
+  (select b1, count(b2) as b2 from B group by b1)
+   on true where a1 = b1 and b2 = a2 and a1 = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $1), =($0, 2))])
+   +- LogicalJoin(condition=[true], joinType=[left])
+      :- LogicalAggregate(group=[{0}], a2=[COUNT($1)])
+      :  +- LogicalProject(a1=[$0], a2=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]])
+      +- LogicalAggregate(group=[{0}], b2=[COUNT($1)])
+         +- LogicalProject(b1=[$0], b2=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[CAST(2 AS INTEGER) AS a1, a2, b1, CAST(b2 AS BIGINT) AS b2])
++- Join(joinType=[InnerJoin], where=[((a1 = b1) AND (b2 = a2))], select=[a1, a2, b1, b2], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey])
+   :- Exchange(distribution=[hash[a1, a2]])
+   :  +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2])
+   :     +- Exchange(distribution=[hash[a1]])
+   :        +- Calc(select=[CAST(2 AS INTEGER) AS a1, a2], where=[(a1 = 2)])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3])
+   +- Exchange(distribution=[hash[b1, b2]])
+      +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2])
+         +- Exchange(distribution=[hash[b1]])
+            +- Calc(select=[CAST(2 AS INTEGER) AS b1, b2], where=[(b1 = 2)])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testLeftJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+  (select a1, count(a2) as a2 from A group by a1)
+   left join
+  (select b1, count(b2) as b2 from B group by b1)
+   on a1 = b1 and a2 = b2 and a1 = 2 and b2 = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($0, 2), =($3, 1))], joinType=[left])
+   :- LogicalAggregate(group=[{0}], a2=[COUNT($1)])
+   :  +- LogicalProject(a1=[$0], a2=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]])
+   +- LogicalAggregate(group=[{0}], b2=[COUNT($1)])
+      +- LogicalProject(b1=[$0], b2=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Join(joinType=[LeftOuterJoin], where=[((a1 = b1) AND (a2 = 1) AND (a1 = 2))], select=[a1, a2, b1, b2], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey])
+:- Exchange(distribution=[hash[a1]])
+:  +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2])
+:     +- Exchange(distribution=[hash[a1]])
+:        +- Calc(select=[a1, a2])
+:           +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3])
++- Exchange(distribution=[hash[b1]])
+   +- Calc(select=[b1, b2], where=[(b2 = 1)])
+      +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2])
+         +- Exchange(distribution=[hash[b1]])
+            +- Calc(select=[b1, b2])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3])
 ]]>
     </Resource>
   </TestCase>
@@ -1019,31 +1180,6 @@ Calc(select=[b, y])
    +- Exchange(distribution=[hash[z]])
       +- Calc(select=[y, z])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z])
-]]>
-    </Resource>
-  </TestCase>
-  <TestCase name="testRightOuterJoinEquiPred">
-    <Resource name="sql">
-      <![CDATA[SELECT b, y FROM t RIGHT OUTER JOIN s ON a = z]]>
-    </Resource>
-    <Resource name="ast">
-      <![CDATA[
-LogicalProject(b=[$1], y=[$4])
-+- LogicalJoin(condition=[=($0, $5)], joinType=[right])
-   :- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]])
-   +- LogicalTableScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]])
-]]>
-    </Resource>
-    <Resource name="optimized exec plan">
-      <![CDATA[
-Calc(select=[b, y])
-+- Join(joinType=[RightOuterJoin], where=[(a = z)], select=[a, b, y, z], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
-   :- Exchange(distribution=[hash[a]])
-   :  +- Calc(select=[a, b])
-   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
-   +- Exchange(distribution=[hash[z]])
-      +- Calc(select=[y, z])
-         +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z])
 ]]>
     </Resource>
   </TestCase>
@@ -1102,6 +1238,147 @@ Join(joinType=[RightOuterJoin], where=[=(a1, b1)], select=[a1, b1], leftInputSpe
       +- Exchange(distribution=[hash[b1]], changelogMode=[I])
          +- Calc(select=[b1, b2], changelogMode=[I])
             +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithEqualPkNonEqui">
+    <Resource name="sql">
+      <![CDATA[SELECT a1, b1 FROM (SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1) RIGHT JOIN (SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1) ON a1 = b1 AND a2 > b2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$1], b1=[$3])
++- LogicalJoin(condition=[AND(=($1, $3), >($0, $2))], joinType=[right])
+   :- LogicalProject(a2=[$1], a1=[$0])
+   :  +- LogicalAggregate(group=[{0}], a2=[SUM($1)])
+   :     +- LogicalProject(a1=[$0], a2=[$1])
+   :        +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]])
+   +- LogicalProject(b2=[$1], b1=[$0])
+      +- LogicalAggregate(group=[{0}], b2=[SUM($1)])
+         +- LogicalProject(b1=[$0], b2=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1], changelogMode=[I,UA,D])
++- Join(joinType=[RightOuterJoin], where=[AND(=(a1, b1), >(a2, b2))], select=[a2, a1, b2, b1], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey], changelogMode=[I,UA,D])
+   :- Exchange(distribution=[hash[a1]], changelogMode=[I,UA])
+   :  +- Calc(select=[a2, a1], changelogMode=[I,UA])
+   :     +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UA])
+   :        +- Exchange(distribution=[hash[a1]], changelogMode=[I])
+   :           +- Calc(select=[a1, a2], changelogMode=[I])
+   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I])
+   +- Exchange(distribution=[hash[b1]], changelogMode=[I,UA])
+      +- Calc(select=[b2, b1], changelogMode=[I,UA])
+         +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UA])
+            +- Exchange(distribution=[hash[b1]], changelogMode=[I])
+               +- Calc(select=[b1, b2], changelogMode=[I])
+                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithFilterPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+  (select a1, count(a2) as a2 from A group by a1)
+   right join
+  (select b1, count(b2) as b2 from B group by b1)
+   on true where a1 = b1 and a2 = b2 and b1 = 2
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $3), =($2, 2))])
+   +- LogicalJoin(condition=[true], joinType=[right])
+      :- LogicalAggregate(group=[{0}], a2=[COUNT($1)])
+      :  +- LogicalProject(a1=[$0], a2=[$1])
+      :     +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]])
+      +- LogicalAggregate(group=[{0}], b2=[COUNT($1)])
+         +- LogicalProject(b1=[$0], b2=[$1])
+            +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[a1, CAST(a2 AS BIGINT) AS a2, CAST(2 AS INTEGER) AS b1, b2])
++- Join(joinType=[InnerJoin], where=[((a1 = b1) AND (a2 = b2))], select=[a1, a2, b1, b2], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey])
+   :- Exchange(distribution=[hash[a1, a2]])
+   :  +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2])
+   :     +- Exchange(distribution=[hash[a1]])
+   :        +- Calc(select=[CAST(2 AS INTEGER) AS a1, a2], where=[(a1 = 2)])
+   :           +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3])
+   +- Exchange(distribution=[hash[b1, b2]])
+      +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2])
+         +- Exchange(distribution=[hash[b1]])
+            +- Calc(select=[CAST(2 AS INTEGER) AS b1, b2], where=[(b1 = 2)])
+               +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightOuterJoinEquiPred">
+    <Resource name="sql">
+      <![CDATA[SELECT b, y FROM t RIGHT OUTER JOIN s ON a = z]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(b=[$1], y=[$4])
++- LogicalJoin(condition=[=($0, $5)], joinType=[right])
+   :- LogicalTableScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Calc(select=[b, y])
++- Join(joinType=[RightOuterJoin], where=[(a = z)], select=[a, b, y, z], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
+   :- Exchange(distribution=[hash[a]])
+   :  +- Calc(select=[a, b])
+   :     +- LegacyTableSourceScan(table=[[default_catalog, default_database, t, source: [TestTableSource(a, b, c)]]], fields=[a, b, c])
+   +- Exchange(distribution=[hash[z]])
+      +- Calc(select=[y, z])
+         +- LegacyTableSourceScan(table=[[default_catalog, default_database, s, source: [TestTableSource(x, y, z)]]], fields=[x, y, z])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testRightJoinWithJoinConditionPushDown">
+    <Resource name="sql">
+      <![CDATA[
+SELECT * FROM
+ (select a1, count(a2) as a2 from A group by a1)
+   right join
+ (select b1, count(b2) as b2 from B group by b1)
+   on a1 = b1 and a2 = b2 and b1 = 2 and a2 = 1
+]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], a2=[$1], b1=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), =($1, $3), =($2, 2), =($1, 1))], joinType=[right])
+   :- LogicalAggregate(group=[{0}], a2=[COUNT($1)])
+   :  +- LogicalProject(a1=[$0], a2=[$1])
+   :     +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]])
+   +- LogicalAggregate(group=[{0}], b2=[COUNT($1)])
+      +- LogicalProject(b1=[$0], b2=[$1])
+         +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]])
+]]>
+    </Resource>
+    <Resource name="optimized exec plan">
+      <![CDATA[
+Join(joinType=[RightOuterJoin], where=[((a1 = b1) AND (1 = b2) AND (b1 = 2))], select=[a1, a2, b1, b2], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey])
+:- Exchange(distribution=[hash[a1]])
+:  +- Calc(select=[a1, a2], where=[(a2 = 1)])
+:     +- GroupAggregate(groupBy=[a1], select=[a1, COUNT(a2) AS a2])
+:        +- Exchange(distribution=[hash[a1]])
+:           +- Calc(select=[a1, a2])
+:              +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3])
++- Exchange(distribution=[hash[b1]])
+   +- GroupAggregate(groupBy=[b1], select=[b1, COUNT(b2) AS b2])
+      +- Exchange(distribution=[hash[b1]])
+         +- Calc(select=[b1, b2])
+            +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3])
 ]]>
     </Resource>
   </TestCase>
@@ -1236,43 +1513,6 @@ Calc(select=[a1, b1], changelogMode=[I,UA,D])
    +- Exchange(distribution=[hash[b1]], changelogMode=[I])
       +- Calc(select=[b1, b2], changelogMode=[I])
          +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I])
-]]>
-    </Resource>
-  </TestCase>
-  <TestCase name="testRightJoinWithEqualPkNonEqui">
-    <Resource name="sql">
-      <![CDATA[SELECT a1, b1 FROM (SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1) RIGHT JOIN (SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1) ON a1 = b1 AND a2 > b2]]>
-    </Resource>
-    <Resource name="ast">
-      <![CDATA[
-LogicalProject(a1=[$1], b1=[$3])
-+- LogicalJoin(condition=[AND(=($1, $3), >($0, $2))], joinType=[right])
-   :- LogicalProject(a2=[$1], a1=[$0])
-   :  +- LogicalAggregate(group=[{0}], a2=[SUM($1)])
-   :     +- LogicalProject(a1=[$0], a2=[$1])
-   :        +- LogicalTableScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]])
-   +- LogicalProject(b2=[$1], b1=[$0])
-      +- LogicalAggregate(group=[{0}], b2=[SUM($1)])
-         +- LogicalProject(b1=[$0], b2=[$1])
-            +- LogicalTableScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]])
-]]>
-    </Resource>
-    <Resource name="optimized rel plan">
-      <![CDATA[
-Calc(select=[a1, b1], changelogMode=[I,UA,D])
-+- Join(joinType=[RightOuterJoin], where=[AND(=(a1, b1), >(a2, b2))], select=[a2, a1, b2, b1], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey], changelogMode=[I,UA,D])
-   :- Exchange(distribution=[hash[a1]], changelogMode=[I,UA])
-   :  +- Calc(select=[a2, a1], changelogMode=[I,UA])
-   :     +- GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2], changelogMode=[I,UA])
-   :        +- Exchange(distribution=[hash[a1]], changelogMode=[I])
-   :           +- Calc(select=[a1, a2], changelogMode=[I])
-   :              +- LegacyTableSourceScan(table=[[default_catalog, default_database, A, source: [TestTableSource(a1, a2, a3)]]], fields=[a1, a2, a3], changelogMode=[I])
-   +- Exchange(distribution=[hash[b1]], changelogMode=[I,UA])
-      +- Calc(select=[b2, b1], changelogMode=[I,UA])
-         +- GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2], changelogMode=[I,UA])
-            +- Exchange(distribution=[hash[b1]], changelogMode=[I])
-               +- Calc(select=[b1, b2], changelogMode=[I])
-                  +- LegacyTableSourceScan(table=[[default_catalog, default_database, B, source: [TestTableSource(b1, b2, b3)]]], fields=[b1, b2, b3], changelogMode=[I])
 ]]>
     </Resource>
   </TestCase>
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
index b0a89b1390b..bab7c14257f 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
@@ -28,7 +28,6 @@ class BroadcastHashJoinTest extends JoinTestBase {
   def before(): Unit = {
     util.tableEnv.getConfig
       .set(OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(Long.MaxValue))
-    Long.box(10)
     util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, ShuffleHashJoin")
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinTestBase.scala
index 503234ecdd8..795e55a6ed2 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinTestBase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/JoinTestBase.scala
@@ -210,4 +210,70 @@ abstract class JoinTestBase extends TableTestBase {
          """.stripMargin
     util.verifyExecPlan(sql)
   }
+
+  @Test
+  def testInnerJoinWithFilterPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |   (select a, count(b) as b from MyTable1 group by a)
+                          |   join
+                          |   (select d, count(e) as e from MyTable2 group by d)
+                          |   on true where a = d and b = e and d = 2
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testInnerJoinWithJoinConditionPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |   (select a, count(b) as b from MyTable1 group by a)
+                          |   join
+                          |   (select d, count(e) as e from MyTable2 group by d)
+                          |   on a = d and b = e and d = 2 and b = 1
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testLeftJoinWithFilterPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |   (select a, count(b) as b from MyTable1 group by a)
+                          |   left join
+                          |   (select d, count(e) as e from MyTable2 group by d)
+                          |   on true where a = d and b = e and a = 2
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testLeftJoinWithJoinConditionPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |   (select a, count(b) as b from MyTable1 group by a)
+                          |   left join
+                          |   (select d, count(e) as e from MyTable2 group by d)
+                          |   on a = d and b = e and a = 2 and e = 1
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testRightJoinWithFilterPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |   (select a, count(b) as b from MyTable1 group by a)
+                          |   right join
+                          |   (select d, count(e) as e from MyTable2 group by d)
+                          |   on true where a = d and b = e and d = 2
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testRightJoinWithJoinConditionPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |   (select a, count(b) as b from MyTable1 group by a)
+                          |   right join
+                          |   (select d, count(e) as e from MyTable2 group by d)
+                          |   on a = d and b = e and d = 2 and b = 1
+                          |""".stripMargin)
+  }
 }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
index dc2500d0242..b76f098897c 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
@@ -18,7 +18,7 @@
 package org.apache.flink.table.planner.plan.batch.sql.join
 
 import org.apache.flink.table.api.TableException
-import org.apache.flink.table.api.config.ExecutionConfigOptions
+import org.apache.flink.table.api.config.{ExecutionConfigOptions, OptimizerConfigOptions}
 
 import org.junit.{Before, Test}
 
@@ -26,6 +26,8 @@ class ShuffledHashJoinTest extends JoinTestBase {
 
   @Before
   def before(): Unit = {
+    util.tableEnv.getConfig.getConfiguration
+      .setLong(OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, 1L)
     util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, BroadcastHashJoin")
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.scala
deleted file mode 100644
index c019fad4f08..00000000000
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkFilterJoinRuleTest.scala
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * 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.api.scala._
-import org.apache.flink.table.api._
-import org.apache.flink.table.planner.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase}
-
-import org.apache.calcite.plan.hep.HepMatchOrder
-import org.apache.calcite.rel.rules.CoreRules
-import org.apache.calcite.tools.RuleSets
-import org.junit.{Before, Test}
-
-/** Tests for [[org.apache.calcite.rel.rules.FilterJoinRule]]. */
-class FlinkFilterJoinRuleTest extends TableTestBase {
-  private val util = batchTestUtil()
-
-  @Before
-  def setup(): Unit = {
-    util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
-    calciteConfig.getBatchProgram.get.addLast(
-      "rules",
-      FlinkHepRuleSetProgramBuilder.newBuilder
-        .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
-        .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
-        .add(
-          RuleSets.ofList(
-            CoreRules.FILTER_PROJECT_TRANSPOSE,
-            CoreRules.FILTER_INTO_JOIN,
-            CoreRules.JOIN_CONDITION_PUSH))
-        .build()
-    )
-
-    util.addTableSource[(Int, Long)]("leftT", 'a, 'b)
-    util.addTableSource[(Int, Long)]("rightT", 'c, 'd)
-  }
-
-  @Test
-  def testFilterPushDownLeftSemi1(): Unit = {
-    val sqlQuery =
-      "SELECT * FROM (SELECT * FROM leftT WHERE a IN (SELECT c FROM rightT)) T WHERE T.b > 2"
-    util.verifyRelPlan(sqlQuery)
-  }
-
-  @Test
-  def testFilterPushDownLeftSemi2(): Unit = {
-    val sqlQuery =
-      "SELECT * FROM (SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT)) T WHERE T.b > 2"
-    util.verifyRelPlan(sqlQuery)
-  }
-
-  @Test
-  def testFilterPushDownLeftSemi3(): Unit = {
-    val sqlQuery =
-      """
-        |SELECT * FROM (SELECT * FROM leftT WHERE EXISTS
-        |    (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2
-      """.stripMargin
-    util.verifyRelPlan(sqlQuery)
-  }
-
-  @Test
-  def testJoinConditionPushDownLeftSemi1(): Unit = {
-    util.verifyRelPlan("SELECT * FROM leftT WHERE a IN (SELECT c FROM rightT WHERE b > 2)")
-  }
-
-  @Test
-  def testJoinConditionPushDownLeftSemi2(): Unit = {
-    util.verifyRelPlan("SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT WHERE b > 2)")
-  }
-
-  @Test
-  def testJoinConditionPushDownLeftSemi3(): Unit = {
-    util.verifyRelPlan(
-      "SELECT * FROM leftT WHERE EXISTS (SELECT * FROM rightT WHERE a = c AND b > 2)")
-  }
-
-  @Test
-  def testFilterPushDownLeftAnti1(): Unit = {
-    val sqlQuery =
-      """
-        |SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN
-        |    (SELECT c FROM rightT WHERE c < 3)) T WHERE T.b > 2
-      """.stripMargin
-    util.verifyRelPlan(sqlQuery)
-  }
-
-  @Test
-  def testFilterPushDownLeftAnti2(): Unit = {
-    val sqlQuery =
-      """
-        |SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS
-        |    (SELECT * FROM rightT where c > 10)) T WHERE T.b > 2
-      """.stripMargin
-    util.verifyRelPlan(sqlQuery)
-  }
-
-  @Test
-  def testFilterPushDownLeftAnti3(): Unit = {
-    val sqlQuery =
-      """
-        |SELECT * FROM (SELECT * FROM leftT WHERE a NOT IN
-        |(SELECT c FROM rightT WHERE b = d AND c < 3)) T WHERE T.b > 2
-      """.stripMargin
-    util.verifyRelPlan(sqlQuery)
-  }
-
-  @Test
-  def testFilterPushDownLeftAnti4(): Unit = {
-    val sqlQuery =
-      """
-        |SELECT * FROM (SELECT * FROM leftT WHERE NOT EXISTS
-        |    (SELECT * FROM rightT WHERE a = c)) T WHERE T.b > 2
-      """.stripMargin
-    util.verifyRelPlan(sqlQuery)
-  }
-
-  @Test
-  def testJoinConditionPushDownLeftAnti1(): Unit = {
-    util.verifyRelPlan("SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE b > 2)")
-  }
-
-  @Test
-  def testJoinConditionPushDownLeftAnti2(): Unit = {
-    util.verifyRelPlan("SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT WHERE b > 2)")
-  }
-
-  @Test
-  def testJoinConditionPushDownLeftAnti3(): Unit = {
-    val sqlQuery = "SELECT * FROM leftT WHERE a NOT IN (SELECT c FROM rightT WHERE b = d AND b > 1)"
-    util.verifyRelPlan(sqlQuery)
-  }
-
-  @Test
-  def testJoinConditionPushDownLeftAnti4(): Unit = {
-    val sqlQuery =
-      "SELECT * FROM leftT WHERE NOT EXISTS (SELECT * FROM rightT WHERE a = c AND b > 2)"
-    util.verifyRelPlan(sqlQuery)
-  }
-
-}
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala
index 7c01ea6c907..fd275f22911 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala
@@ -32,17 +32,17 @@ class JoinTest extends TableTestBase {
   util.addTableSource[(Long, String, Int)]("s", 'x, 'y, 'z)
 
   @Test
-  def testDependentConditionDerivationInnerJoin: Unit = {
+  def testDependentConditionDerivationInnerJoin(): Unit = {
     util.verifyExecPlan("SELECT a1, b1 FROM A JOIN B ON (a1 = 1 AND b1 = 1) OR (a2 = 2 AND b2 = 2)")
   }
 
   @Test
-  def testDependentConditionDerivationInnerJoinWithTrue: Unit = {
+  def testDependentConditionDerivationInnerJoinWithTrue(): Unit = {
     util.verifyExecPlan("SELECT a1, b1 FROM A JOIN B ON (a1 = 1 AND b1 = 1) OR (a2 = 2 AND true)")
   }
 
   @Test
-  def testDependentConditionDerivationInnerJoinWithNull: Unit = {
+  def testDependentConditionDerivationInnerJoinWithNull(): Unit = {
     util.verifyExecPlan("SELECT * FROM t JOIN s ON (a = 1 AND x = 1) OR (a = 2 AND y is null)")
   }
 
@@ -504,4 +504,71 @@ class JoinTest extends TableTestBase {
       ExplainDetail.CHANGELOG_MODE
     )
   }
+
+  @Test
+  def testInnerJoinWithFilterPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |   (select a1, count(a2) as a2 from A group by a1)
+                          |   join
+                          |   (select b1, count(b2) as b2 from B group by b1)
+                          |   on true where a1 = b1 and a2 = b2 and b1 = 2
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testInnerJoinWithJoinConditionPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |  (select a1, count(a2) as a2 from A group by a1)
+                          |   join
+                          |  (select b1, count(b2) as b2 from B group by b1)
+                          |   on true where a1 = b1 and a2 = b2 and b1 = 2 and a2 = 1
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testLeftJoinWithFilterPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |  (select a1, count(a2) as a2 from A group by a1)
+                          |   left join
+                          |  (select b1, count(b2) as b2 from B group by b1)
+                          |   on true where a1 = b1 and b2 = a2 and a1 = 2
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testLeftJoinWithJoinConditionPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |  (select a1, count(a2) as a2 from A group by a1)
+                          |   left join
+                          |  (select b1, count(b2) as b2 from B group by b1)
+                          |   on a1 = b1 and a2 = b2 and a1 = 2 and b2 = 1
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testRightJoinWithFilterPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          |  (select a1, count(a2) as a2 from A group by a1)
+                          |   right join
+                          |  (select b1, count(b2) as b2 from B group by b1)
+                          |   on true where a1 = b1 and a2 = b2 and b1 = 2
+                          |""".stripMargin)
+  }
+
+  @Test
+  def testRightJoinWithJoinConditionPushDown(): Unit = {
+    util.verifyExecPlan("""
+                          |SELECT * FROM
+                          | (select a1, count(a2) as a2 from A group by a1)
+                          |   right join
+                          | (select b1, count(b2) as b2 from B group by b1)
+                          |   on a1 = b1 and a2 = b2 and b1 = 2 and a2 = 1
+                          |""".stripMargin)
+  }
+
 }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
index 0c954918a29..0eb199a0f35 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
@@ -1293,6 +1293,146 @@ class JoinITCase(expectedJoinType: JoinType) extends BatchTestBase {
       Seq(row("Hi", "Hallo"), row("Hello", "Hallo Welt"), row("Hello world", "Hallo Welt"))
     )
   }
+
+  @Test
+  def testJoinWithFilterPushDown(): Unit = {
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where a >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  left join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where a >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  left join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where c >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  right join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where a >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  right join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where c >= 2
+        |""".stripMargin,
+      Seq(
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, null, null, 4, 1.0, 1))
+    )
+  }
+
+  @Test
+  def testJoinWithJoinConditionPushDown(): Unit = {
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and a >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  left join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and a >= 2
+        |""".stripMargin,
+      Seq(
+        row(1, 2.0, 2, null, null, null),
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, 5.0, 2, null, null, null))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  left join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and c >= 2
+        |""".stripMargin,
+      Seq(
+        row(1, 2.0, 2, null, null, null),
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, 5.0, 2, null, null, null))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  right join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and a >= 2
+        |""".stripMargin,
+      Seq(
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, null, null, 4, 1.0, 1),
+        row(null, null, null, null, 5.0, 2))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  right join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and c >= 2
+        |""".stripMargin,
+      Seq(
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, null, null, 4, 1.0, 1),
+        row(null, null, null, null, 5.0, 2))
+    )
+  }
 }
 
 object JoinITCase {
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/JoinITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/JoinITCase.scala
index efc05be0f7c..86863160348 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/JoinITCase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/JoinITCase.scala
@@ -22,7 +22,9 @@ import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.table.api._
 import org.apache.flink.table.api.bridge.scala._
 import org.apache.flink.table.planner.expressions.utils.FuncWithOpen
+import org.apache.flink.table.planner.factories.TestValuesTableFactory
 import org.apache.flink.table.planner.runtime.utils._
+import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.StateBackendMode
 import org.apache.flink.types.Row
 
@@ -66,6 +68,29 @@ class JoinITCase(state: StateBackendMode) extends StreamingWithStateTestBase(sta
       .toTable(tEnv, 'b1, 'b2, 'b3, 'b4, 'b5)
     tEnv.registerTable("A", tableA)
     tEnv.registerTable("B", tableB)
+
+    val dataId1 = TestValuesTableFactory.registerData(TestData.data2_1)
+    tEnv.executeSql(s"""
+                       |create table l (
+                       |  a int,
+                       |  b double
+                       |) with (
+                       |  'connector' = 'values',
+                       |  'data-id' = '$dataId1',
+                       |  'bounded' = 'true'
+                       |)
+                       |""".stripMargin)
+    val dataId2 = TestValuesTableFactory.registerData(TestData.data2_2)
+    tEnv.executeSql(s"""
+                       |create table r (
+                       |  c int,
+                       |  d double
+                       |) with (
+                       |  'connector' = 'values',
+                       |  'data-id' = '$dataId2',
+                       |  'bounded' = 'true'
+                       |)
+                       |""".stripMargin)
   }
 
   // Tests for inner join.
@@ -1415,4 +1440,158 @@ class JoinITCase(state: StateBackendMode) extends StreamingWithStateTestBase(sta
     val expected = Seq("Hi,Hallo", "Hello,Hallo Welt", "Hello world,Hallo Welt")
     assertEquals(expected.sorted, sink.getRetractResults.sorted)
   }
+
+  @Test
+  def testJoinWithFilterPushDown(): Unit = {
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where a >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  left join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where a >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  left join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where c >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  right join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where a >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  right join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 where c >= 2
+        |""".stripMargin,
+      Seq(
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, null, null, 4, 1.0, 1))
+    )
+  }
+
+  @Test
+  def testJoinWithJoinConditionPushDown(): Unit = {
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and a >= 2
+        |""".stripMargin,
+      Seq(row(2, 1.0, 2, 2, 3.0, 2), row(3, 3.0, 1, 3, 2.0, 1), row(6, null, 1, 6, null, 1))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  left join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and a >= 2
+        |""".stripMargin,
+      Seq(
+        row(1, 2.0, 2, null, null, null),
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, 5.0, 2, null, null, null))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  left join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and c >= 2
+        |""".stripMargin,
+      Seq(
+        row(1, 2.0, 2, null, null, null),
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, 5.0, 2, null, null, null))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  right join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and a >= 2
+        |""".stripMargin,
+      Seq(
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, null, null, 4, 1.0, 1),
+        row(null, null, null, null, 5.0, 2))
+    )
+
+    checkResult(
+      """
+        |select * from
+        |  (select a, max(b) b, count(*) c1 from l group by a)
+        |  right join
+        |  (select c, max(d) d, count(*) c2 from r group by c)
+        |  on a = c and c1 = c2 and c >= 2
+        |""".stripMargin,
+      Seq(
+        row(2, 1.0, 2, 2, 3.0, 2),
+        row(3, 3.0, 1, 3, 2.0, 1),
+        row(6, null, 1, 6, null, 1),
+        row(null, null, null, 4, 1.0, 1),
+        row(null, null, null, null, 5.0, 2))
+    )
+  }
+
+  private def checkResult(sql: String, expected: Seq[Row]): Unit = {
+    val sink = new TestingRetractSink
+    tEnv.sqlQuery(sql).toRetractStream[Row].addSink(sink).setParallelism(1)
+    env.execute()
+
+    val expectedResult = expected
+      .map(
+        r => {
+          (0 until r.getArity).map(i => r.getField(i)).mkString(",")
+        })
+      .sorted
+    assertEquals(expectedResult, sink.getRetractResults.sorted)
+  }
 }