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/08 03:56:17 UTC

[flink] branch master updated (d137c12877f -> f794d72903c)

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

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


    from d137c12877f [FLINK-28688][python] Support DataStream PythonWindowOperator in Thread Mode
     new d17dcbd43b2 [FLINK-28678][table-planner] Copy some nodes from calcite, and need revert when upgrade calcite-1.31
     new 4e46602342f [FLINK-28678][table-planner] Support SQL hint for Filter, SetOp, and etc. Mainly cherry-pick from [calcite-5107]
     new 53f1a66f67c [FLINK-28682][table-planner] Copy SqlToRelConverter from calcite
     new f794d72903c [FLINK-28682][table-planner] Support join hint in batch rules

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../org/apache/calcite/rel/core/Correlate.java     |  278 +
 .../java/org/apache/calcite/rel/core/Filter.java   |  201 +
 .../org/apache/calcite/rel/core/Intersect.java     |   72 +
 .../java/org/apache/calcite/rel/core/Minus.java    |   69 +
 .../java/org/apache/calcite/rel/core/SetOp.java    |  158 +
 .../java/org/apache/calcite/rel/core/Sort.java     |  232 +
 .../java/org/apache/calcite/rel/core/Union.java    |   79 +
 .../java/org/apache/calcite/rel/core/Values.java   |  249 +
 .../java/org/apache/calcite/rel/core/Window.java   |  471 ++
 .../apache/calcite/rel/hint/HintPredicates.java    |  133 +
 .../calcite/rel/hint/NodeTypeHintPredicate.java    |  107 +
 .../calcite/rel/logical/LogicalCorrelate.java      |  188 +
 .../apache/calcite/rel/logical/LogicalFilter.java  |  175 +
 .../calcite/rel/logical/LogicalIntersect.java      |   97 +
 .../apache/calcite/rel/logical/LogicalMinus.java   |  101 +
 .../apache/calcite/rel/logical/LogicalSort.java    |  109 +
 .../apache/calcite/rel/logical/LogicalUnion.java   |  100 +
 .../apache/calcite/rel/logical/LogicalValues.java  |  169 +
 .../apache/calcite/rel/logical/LogicalWindow.java  |  399 ++
 .../apache/calcite/sql2rel/RelDecorrelator.java    |   11 +
 .../apache/calcite/sql2rel/SqlToRelConverter.java  | 6034 ++++++++++++++++++++
 ...ClearJoinHintWithInvalidPropagationShuttle.java |  179 +
 .../planner/calcite/RelTimeIndicatorConverter.java |    3 +-
 .../table/planner/hint/FlinkHintStrategies.java    |   54 +
 .../flink/table/planner/hint/FlinkHints.java       |  101 +
 .../flink/table/planner/hint/JoinStrategy.java     |   87 +
 .../optimize/ClearQueryBlockAliasResolver.java     |   71 +
 .../planner/plan/optimize/JoinHintResolver.java    |  237 +
 .../StreamNonDeterministicUpdatePlanVisitor.java   |    9 +-
 .../planner/calcite/FlinkLogicalRelFactories.scala |    2 +-
 .../table/planner/calcite/FlinkPlannerImpl.scala   |   49 +-
 .../table/planner/plan/nodes/calcite/Sink.scala    |    2 +-
 .../plan/nodes/logical/FlinkLogicalJoin.scala      |   21 +-
 .../BatchCommonSubGraphBasedOptimizer.scala        |   15 +-
 .../logical/SplitPythonConditionFromJoinRule.scala |    1 +
 .../TemporalJoinRewriteWithUniqueKeyRule.scala     |    7 +-
 .../physical/batch/BatchPhysicalHashJoinRule.scala |  111 +-
 .../physical/batch/BatchPhysicalJoinRuleBase.scala |  258 +-
 .../batch/BatchPhysicalNestedLoopJoinRule.scala    |   53 +-
 .../batch/BatchPhysicalSingleRowJoinRule.scala     |   23 +-
 .../batch/BatchPhysicalSortMergeJoinRule.scala     |   12 +-
 .../table/planner/plan/utils/FlinkRelOptUtil.scala |   10 +-
 .../planner/plan/utils/RelTreeWriterImpl.scala     |   36 +-
 ...rJoinHintWithInvalidPropagationShuttleTest.java |  225 +
 .../sql/join/joinhint/BroadcastJoinHintTest.java   |   35 +
 .../batch/sql/join/joinhint/JoinHintTestBase.java  |  726 +++
 .../sql/join/joinhint/NestLoopJoinHintTest.java    |   35 +
 .../sql/join/joinhint/ShuffleHashJoinHintTest.java |   35 +
 .../join/joinhint/ShuffleMergeJoinHintTest.java    |   35 +
 .../optimize/ClearQueryBlockAliasResolverTest.java |  104 +
 .../plan/optimize/JoinHintResolverTest.java        |  101 +
 ...arJoinHintWithInvalidPropagationShuttleTest.xml |  150 +
 .../plan/batch/sql/join/ShuffledHashJoinTest.xml   |   32 +
 .../sql/join/joinhint/BroadcastJoinHintTest.xml    | 1210 ++++
 .../sql/join/joinhint/NestLoopJoinHintTest.xml     | 1206 ++++
 .../sql/join/joinhint/ShuffleHashJoinHintTest.xml  | 1246 ++++
 .../sql/join/joinhint/ShuffleMergeJoinHintTest.xml | 1246 ++++
 .../optimize/ClearQueryBlockAliasResolverTest.xml  |  702 +++
 .../planner/plan/optimize/JoinHintResolverTest.xml |  702 +++
 .../plan/batch/sql/join/ShuffledHashJoinTest.scala |    8 -
 .../rules/logical/subquery/SubQueryTestBase.scala  |    4 +-
 .../flink/table/planner/utils/TableTestBase.scala  |   72 +-
 62 files changed, 18482 insertions(+), 165 deletions(-)
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Correlate.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Filter.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Intersect.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Minus.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/SetOp.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Sort.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Union.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Values.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Window.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/HintPredicates.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/NodeTypeHintPredicate.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalSort.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttle.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/JoinStrategy.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolver.java
 create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java
 create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java
 create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.java
 create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/JoinHintTestBase.java
 create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.java
 create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.java
 create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.java
 create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java
 create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java
 create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.xml
 create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.xml
 create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.xml
 create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.xml
 create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.xml
 create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml
 create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.xml


[flink] 03/04: [FLINK-28682][table-planner] Copy SqlToRelConverter from calcite

Posted by go...@apache.org.
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

commit 53f1a66f67ca1414e9b8f833f79466aa6b2110f5
Author: xuyang <xy...@163.com>
AuthorDate: Tue Jul 26 14:39:12 2022 +0800

    [FLINK-28682][table-planner] Copy SqlToRelConverter from calcite
    
    Due to Calcite will expand the whole SQL Rel Node tree that contains query block, but sometimes the query block should be perceived. A new node QueryBlockAlias will be added into the RelNode tree to isolate the query block and some hints (like join hints) will not be propagated into the query block.
    
    This pr is just to copy SqlToRelConverter from calcite and doesn't do any modification.
    
    This closes #20359
---
 .../apache/calcite/sql2rel/SqlToRelConverter.java  | 5975 ++++++++++++++++++++
 1 file changed, 5975 insertions(+)

diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
new file mode 100644
index 00000000000..11414792b74
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -0,0 +1,5975 @@
+/*
+ * 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.calcite.sql2rel;
+
+import org.apache.flink.table.planner.alias.SubQueryAliasNodeClearShuttle;
+import org.apache.flink.table.planner.plan.nodes.calcite.LogicalSubQueryAlias;
+
+import org.apache.calcite.avatica.util.Spaces;
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptSamplingParameters;
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.ViewExpanders;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.prepare.RelOptTableImpl;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelRoot;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Collect;
+import org.apache.calcite.rel.core.CorrelationId;
+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.core.Project;
+import org.apache.calcite.rel.core.RelFactories;
+import org.apache.calcite.rel.core.Sample;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.hint.HintStrategyTable;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.rel.logical.LogicalMatch;
+import org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.calcite.rel.logical.LogicalTableModify;
+import org.apache.calcite.rel.logical.LogicalTableScan;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.stream.Delta;
+import org.apache.calcite.rel.stream.LogicalDelta;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexBuilder;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCallBinding;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexFieldCollation;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexSubQuery;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.rex.RexWindowBounds;
+import org.apache.calcite.schema.ColumnStrategy;
+import org.apache.calcite.schema.ModifiableTable;
+import org.apache.calcite.schema.ModifiableView;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.TranslatableTable;
+import org.apache.calcite.schema.Wrapper;
+import org.apache.calcite.sql.JoinConditionType;
+import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.sql.SqlBasicCall;
+import org.apache.calcite.sql.SqlCall;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.calcite.sql.SqlDelete;
+import org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.calcite.sql.SqlExplainFormat;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlInsert;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlJoin;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlLiteral;
+import org.apache.calcite.sql.SqlMatchRecognize;
+import org.apache.calcite.sql.SqlMerge;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlNodeList;
+import org.apache.calcite.sql.SqlNumericLiteral;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlOrderBy;
+import org.apache.calcite.sql.SqlPivot;
+import org.apache.calcite.sql.SqlSampleSpec;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlSelectKeyword;
+import org.apache.calcite.sql.SqlSetOperator;
+import org.apache.calcite.sql.SqlSnapshot;
+import org.apache.calcite.sql.SqlUnnestOperator;
+import org.apache.calcite.sql.SqlUpdate;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.SqlValuesOperator;
+import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.sql.SqlWith;
+import org.apache.calcite.sql.SqlWithItem;
+import org.apache.calcite.sql.fun.SqlCase;
+import org.apache.calcite.sql.fun.SqlCountAggFunction;
+import org.apache.calcite.sql.fun.SqlInOperator;
+import org.apache.calcite.sql.fun.SqlQuantifyOperator;
+import org.apache.calcite.sql.fun.SqlRowOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.type.TableFunctionReturnTypeInference;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
+import org.apache.calcite.sql.util.SqlVisitor;
+import org.apache.calcite.sql.validate.AggregatingSelectScope;
+import org.apache.calcite.sql.validate.CollectNamespace;
+import org.apache.calcite.sql.validate.DelegatingScope;
+import org.apache.calcite.sql.validate.ListScope;
+import org.apache.calcite.sql.validate.MatchRecognizeScope;
+import org.apache.calcite.sql.validate.ParameterScope;
+import org.apache.calcite.sql.validate.SelectScope;
+import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlNameMatcher;
+import org.apache.calcite.sql.validate.SqlQualified;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
+import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorNamespace;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.sql.validate.SqlValidatorTable;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.calcite.util.ImmutableBeans;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.NumberUtil;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+import org.apache.calcite.util.trace.CalciteTrace;
+import org.slf4j.Logger;
+
+import javax.annotation.Nonnull;
+
+import java.lang.reflect.Type;
+import java.math.BigDecimal;
+import java.util.AbstractList;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.function.Supplier;
+import java.util.function.UnaryOperator;
+import java.util.stream.Collectors;
+
+import static org.apache.calcite.sql.SqlUtil.stripAs;
+
+/**
+ * Converts a SQL parse tree (consisting of {@link SqlNode} objects) into a relational algebra
+ * expression (consisting of {@link RelNode} objects).
+ *
+ * <p>The public entry points are: {@link #convertQuery}, {@link #convertExpression(SqlNode)}.
+ */
+@SuppressWarnings("UnstableApiUsage")
+public class SqlToRelConverter {
+    // ~ Static fields/initializers ---------------------------------------------
+
+    /** Default configuration. */
+    private static final Config CONFIG =
+            ImmutableBeans.create(Config.class)
+                    .withRelBuilderFactory(RelFactories.LOGICAL_BUILDER)
+                    .withRelBuilderConfigTransform(c -> c.withPushJoinCondition(true))
+                    .withHintStrategyTable(HintStrategyTable.EMPTY);
+
+    protected static final Logger SQL2REL_LOGGER = CalciteTrace.getSqlToRelTracer();
+
+    /** Size of the smallest IN list that will be converted to a semijoin to a static table. */
+    public static final int DEFAULT_IN_SUB_QUERY_THRESHOLD = 20;
+
+    @Deprecated // to be removed before 2.0
+    public static final int DEFAULT_IN_SUBQUERY_THRESHOLD = DEFAULT_IN_SUB_QUERY_THRESHOLD;
+
+    // ~ Instance fields --------------------------------------------------------
+
+    protected final SqlValidator validator;
+    protected final RexBuilder rexBuilder;
+    protected final Prepare.CatalogReader catalogReader;
+    protected final RelOptCluster cluster;
+    private SubQueryConverter subQueryConverter;
+    protected final Map<RelNode, Integer> leaves = new HashMap<>();
+    private final List<SqlDynamicParam> dynamicParamSqlNodes = new ArrayList<>();
+    private final SqlOperatorTable opTab;
+    protected final RelDataTypeFactory typeFactory;
+    private final SqlNodeToRexConverter exprConverter;
+    private final HintStrategyTable hintStrategies;
+    private int explainParamCount;
+    public final Config config;
+    private final RelBuilder relBuilder;
+
+    /** Fields used in name resolution for correlated sub-queries. */
+    private final Map<CorrelationId, DeferredLookup> mapCorrelToDeferred = new HashMap<>();
+
+    /**
+     * Stack of names of datasets requested by the <code>
+     * TABLE(SAMPLE(&lt;datasetName&gt;, &lt;query&gt;))</code> construct.
+     */
+    private final Deque<String> datasetStack = new ArrayDeque<>();
+
+    /**
+     * Mapping of non-correlated sub-queries that have been converted to their equivalent constants.
+     * Used to avoid re-evaluating the sub-query if it's already been evaluated.
+     */
+    private final Map<SqlNode, RexNode> mapConvertedNonCorrSubqs = new HashMap<>();
+
+    public final RelOptTable.ViewExpander viewExpander;
+
+    // ~ Constructors -----------------------------------------------------------
+    /**
+     * Creates a converter.
+     *
+     * @param viewExpander Preparing statement
+     * @param validator Validator
+     * @param catalogReader Schema
+     * @param planner Planner
+     * @param rexBuilder Rex builder
+     * @param convertletTable Expression converter
+     */
+    @Deprecated // to be removed before 2.0
+    public SqlToRelConverter(
+            RelOptTable.ViewExpander viewExpander,
+            SqlValidator validator,
+            Prepare.CatalogReader catalogReader,
+            RelOptPlanner planner,
+            RexBuilder rexBuilder,
+            SqlRexConvertletTable convertletTable) {
+        this(
+                viewExpander,
+                validator,
+                catalogReader,
+                RelOptCluster.create(planner, rexBuilder),
+                convertletTable,
+                SqlToRelConverter.config());
+    }
+
+    @Deprecated // to be removed before 2.0
+    public SqlToRelConverter(
+            RelOptTable.ViewExpander viewExpander,
+            SqlValidator validator,
+            Prepare.CatalogReader catalogReader,
+            RelOptCluster cluster,
+            SqlRexConvertletTable convertletTable) {
+        this(
+                viewExpander,
+                validator,
+                catalogReader,
+                cluster,
+                convertletTable,
+                SqlToRelConverter.config());
+    }
+
+    /* Creates a converter. */
+    public SqlToRelConverter(
+            RelOptTable.ViewExpander viewExpander,
+            SqlValidator validator,
+            Prepare.CatalogReader catalogReader,
+            RelOptCluster cluster,
+            SqlRexConvertletTable convertletTable,
+            Config config) {
+        this.viewExpander = viewExpander;
+        this.opTab =
+                (validator == null) ? SqlStdOperatorTable.instance() : validator.getOperatorTable();
+        this.validator = validator;
+        this.catalogReader = catalogReader;
+        this.subQueryConverter = new NoOpSubQueryConverter();
+        this.rexBuilder = cluster.getRexBuilder();
+        this.typeFactory = rexBuilder.getTypeFactory();
+        this.exprConverter = new SqlNodeToRexConverterImpl(convertletTable);
+        this.explainParamCount = 0;
+        this.config = Objects.requireNonNull(config);
+        this.relBuilder =
+                config.getRelBuilderFactory()
+                        .create(cluster, null)
+                        .transform(config.getRelBuilderConfigTransform());
+        this.hintStrategies = config.getHintStrategyTable();
+
+        cluster.setHintStrategies(this.hintStrategies);
+        this.cluster = Objects.requireNonNull(cluster);
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    /** Returns the RelOptCluster in use. */
+    public RelOptCluster getCluster() {
+        return cluster;
+    }
+
+    /** Returns the row-expression builder. */
+    public RexBuilder getRexBuilder() {
+        return rexBuilder;
+    }
+
+    /**
+     * Returns the number of dynamic parameters encountered during translation; this must only be
+     * called after {@link #convertQuery}.
+     *
+     * @return number of dynamic parameters
+     */
+    public int getDynamicParamCount() {
+        return dynamicParamSqlNodes.size();
+    }
+
+    /**
+     * Returns the type inferred for a dynamic parameter.
+     *
+     * @param index 0-based index of dynamic parameter
+     * @return inferred type, never null
+     */
+    public RelDataType getDynamicParamType(int index) {
+        SqlNode sqlNode = dynamicParamSqlNodes.get(index);
+        if (sqlNode == null) {
+            throw Util.needToImplement("dynamic param type inference");
+        }
+        return validator.getValidatedNodeType(sqlNode);
+    }
+
+    /**
+     * Returns the current count of the number of dynamic parameters in an EXPLAIN PLAN statement.
+     *
+     * @param increment if true, increment the count
+     * @return the current count before the optional increment
+     */
+    public int getDynamicParamCountInExplain(boolean increment) {
+        int retVal = explainParamCount;
+        if (increment) {
+            ++explainParamCount;
+        }
+        return retVal;
+    }
+
+    /**
+     * Returns the mapping of non-correlated sub-queries that have been converted to the constants
+     * that they evaluate to.
+     */
+    public Map<SqlNode, RexNode> getMapConvertedNonCorrSubqs() {
+        return mapConvertedNonCorrSubqs;
+    }
+
+    /**
+     * Adds to the current map of non-correlated converted sub-queries the elements from another map
+     * that contains non-correlated sub-queries that have been converted by another
+     * SqlToRelConverter.
+     *
+     * @param alreadyConvertedNonCorrSubqs the other map
+     */
+    public void addConvertedNonCorrSubqs(Map<SqlNode, RexNode> alreadyConvertedNonCorrSubqs) {
+        mapConvertedNonCorrSubqs.putAll(alreadyConvertedNonCorrSubqs);
+    }
+
+    /**
+     * Sets a new SubQueryConverter. To have any effect, this must be called before any convert
+     * method.
+     *
+     * @param converter new SubQueryConverter
+     */
+    public void setSubQueryConverter(SubQueryConverter converter) {
+        subQueryConverter = converter;
+    }
+
+    /**
+     * Sets the number of dynamic parameters in the current EXPLAIN PLAN statement.
+     *
+     * @param explainParamCount number of dynamic parameters in the statement
+     */
+    public void setDynamicParamCountInExplain(int explainParamCount) {
+        assert config.isExplain();
+        this.explainParamCount = explainParamCount;
+    }
+
+    private void checkConvertedType(SqlNode query, RelNode result) {
+        if (query.isA(SqlKind.DML)) {
+            return;
+        }
+        // Verify that conversion from SQL to relational algebra did
+        // not perturb any type information.  (We can't do this if the
+        // SQL statement is something like an INSERT which has no
+        // validator type information associated with its result,
+        // hence the namespace check above.)
+        final List<RelDataTypeField> validatedFields =
+                validator.getValidatedNodeType(query).getFieldList();
+        final RelDataType validatedRowType =
+                validator
+                        .getTypeFactory()
+                        .createStructType(
+                                Pair.right(validatedFields),
+                                SqlValidatorUtil.uniquify(
+                                        Pair.left(validatedFields),
+                                        catalogReader.nameMatcher().isCaseSensitive()));
+
+        final List<RelDataTypeField> convertedFields =
+                result.getRowType().getFieldList().subList(0, validatedFields.size());
+        final RelDataType convertedRowType =
+                validator.getTypeFactory().createStructType(convertedFields);
+
+        if (!RelOptUtil.equal(
+                "validated row type",
+                validatedRowType,
+                "converted row type",
+                convertedRowType,
+                Litmus.IGNORE)) {
+            throw new AssertionError(
+                    "Conversion to relational algebra failed to "
+                            + "preserve datatypes:\n"
+                            + "validated type:\n"
+                            + validatedRowType.getFullTypeString()
+                            + "\nconverted type:\n"
+                            + convertedRowType.getFullTypeString()
+                            + "\nrel:\n"
+                            + RelOptUtil.toString(result));
+        }
+    }
+
+    public RelNode flattenTypes(RelNode rootRel, boolean restructure) {
+        RelStructuredTypeFlattener typeFlattener =
+                new RelStructuredTypeFlattener(
+                        relBuilder,
+                        rexBuilder,
+                        createToRelContext(com.google.common.collect.ImmutableList.of()),
+                        restructure);
+        return typeFlattener.rewrite(rootRel);
+    }
+
+    /**
+     * If sub-query is correlated and decorrelation is enabled, performs decorrelation.
+     *
+     * @param query Query
+     * @param rootRel Root relational expression
+     * @return New root relational expression after decorrelation
+     */
+    public RelNode decorrelate(SqlNode query, RelNode rootRel) {
+        if (!config.isDecorrelationEnabled()) {
+            return rootRel;
+        }
+        final RelNode result = decorrelateQuery(rootRel);
+        if (result != rootRel) {
+            checkConvertedType(query, result);
+        }
+        return result;
+    }
+
+    /**
+     * Walks over a tree of relational expressions, replacing each {@link RelNode} with a 'slimmed
+     * down' relational expression that projects only the fields required by its consumer.
+     *
+     * <p>This may make things easier for the optimizer, by removing crud that would expand the
+     * search space, but is difficult for the optimizer itself to do it, because optimizer rules
+     * must preserve the number and type of fields. Hence, this transform that operates on the
+     * entire tree, similar to the {@link RelStructuredTypeFlattener type-flattening transform}.
+     *
+     * <p>Currently this functionality is disabled in farrago/luciddb; the default implementation of
+     * this method does nothing.
+     *
+     * @param ordered Whether the relational expression must produce results in a particular order
+     *     (typically because it has an ORDER BY at top level)
+     * @param rootRel Relational expression that is at the root of the tree
+     * @return Trimmed relational expression
+     */
+    public RelNode trimUnusedFields(boolean ordered, RelNode rootRel) {
+        // Trim fields that are not used by their consumer.
+        if (isTrimUnusedFields()) {
+            final RelFieldTrimmer trimmer = newFieldTrimmer();
+            final List<RelCollation> collations =
+                    rootRel.getTraitSet().getTraits(RelCollationTraitDef.INSTANCE);
+            rootRel = trimmer.trim(rootRel);
+            if (!ordered
+                    && collations != null
+                    && !collations.isEmpty()
+                    && !collations.equals(
+                            com.google.common.collect.ImmutableList.of(RelCollations.EMPTY))) {
+                final RelTraitSet traitSet =
+                        rootRel.getTraitSet().replace(RelCollationTraitDef.INSTANCE, collations);
+                rootRel = rootRel.copy(traitSet, rootRel.getInputs());
+            }
+            if (SQL2REL_LOGGER.isDebugEnabled()) {
+                SQL2REL_LOGGER.debug(
+                        RelOptUtil.dumpPlan(
+                                "Plan after trimming unused fields",
+                                rootRel,
+                                SqlExplainFormat.TEXT,
+                                SqlExplainLevel.EXPPLAN_ATTRIBUTES));
+            }
+        }
+        return rootRel;
+    }
+
+    /**
+     * Creates a RelFieldTrimmer.
+     *
+     * @return Field trimmer
+     */
+    protected RelFieldTrimmer newFieldTrimmer() {
+        return new RelFieldTrimmer(validator, relBuilder);
+    }
+
+    /**
+     * Converts an unvalidated query's parse tree into a relational expression.
+     *
+     * @param query Query to convert
+     * @param needsValidation Whether to validate the query before converting; <code>false</code> if
+     *     the query has already been validated.
+     * @param top Whether the query is top-level, say if its result will become a JDBC result set;
+     *     <code>false</code> if the query will be part of a view.
+     */
+    public RelRoot convertQuery(SqlNode query, final boolean needsValidation, final boolean top) {
+        if (needsValidation) {
+            query = validator.validate(query);
+        }
+
+        RelNode result = convertQueryRecursive(query, top, null).rel;
+        if (top) {
+            if (isStream(query)) {
+                result = new LogicalDelta(cluster, result.getTraitSet(), result);
+            }
+        }
+        RelCollation collation = RelCollations.EMPTY;
+        if (!query.isA(SqlKind.DML)) {
+            if (isOrdered(query)) {
+                collation = requiredCollation(result);
+            }
+        }
+        checkConvertedType(query, result);
+
+        if (SQL2REL_LOGGER.isDebugEnabled()) {
+            SQL2REL_LOGGER.debug(
+                    RelOptUtil.dumpPlan(
+                            "Plan after converting SqlNode to RelNode",
+                            result,
+                            SqlExplainFormat.TEXT,
+                            SqlExplainLevel.EXPPLAN_ATTRIBUTES));
+        }
+
+        final RelDataType validatedRowType = validator.getValidatedNodeType(query);
+        List<RelHint> hints = new ArrayList<>();
+        if (query.getKind() == SqlKind.SELECT) {
+            final SqlSelect select = (SqlSelect) query;
+            if (select.hasHints()) {
+                hints = SqlUtil.getRelHint(hintStrategies, select.getHints());
+            }
+        }
+        // propagate the hints.
+        result = RelOptUtil.propagateRelHints(result, false);
+        return RelRoot.of(result, validatedRowType, query.getKind())
+                .withCollation(collation)
+                .withHints(hints);
+    }
+
+    private static boolean isStream(SqlNode query) {
+        return query instanceof SqlSelect
+                && ((SqlSelect) query).isKeywordPresent(SqlSelectKeyword.STREAM);
+    }
+
+    public static boolean isOrdered(SqlNode query) {
+        switch (query.getKind()) {
+            case SELECT:
+                return ((SqlSelect) query).getOrderList() != null
+                        && ((SqlSelect) query).getOrderList().size() > 0;
+            case WITH:
+                return isOrdered(((SqlWith) query).body);
+            case ORDER_BY:
+                return ((SqlOrderBy) query).orderList.size() > 0;
+            default:
+                return false;
+        }
+    }
+
+    private RelCollation requiredCollation(RelNode r) {
+        if (r instanceof Sort) {
+            return ((Sort) r).collation;
+        }
+        if (r instanceof Project) {
+            return requiredCollation(((Project) r).getInput());
+        }
+        if (r instanceof Delta) {
+            return requiredCollation(((Delta) r).getInput());
+        }
+        throw new AssertionError();
+    }
+
+    /** Converts a SELECT statement's parse tree into a relational expression. */
+    public RelNode convertSelect(SqlSelect select, boolean top) {
+        final SqlValidatorScope selectScope = validator.getWhereScope(select);
+        final Blackboard bb = createBlackboard(selectScope, null, top);
+        convertSelectImpl(bb, select);
+        return bb.root;
+    }
+
+    /** Factory method for creating translation workspace. */
+    protected Blackboard createBlackboard(
+            SqlValidatorScope scope, Map<String, RexNode> nameToNodeMap, boolean top) {
+        return new Blackboard(scope, nameToNodeMap, top);
+    }
+
+    /** Implementation of {@link #convertSelect(SqlSelect, boolean)}; derived class may override. */
+    protected void convertSelectImpl(final Blackboard bb, SqlSelect select) {
+        convertFrom(bb, select.getFrom());
+        convertWhere(bb, select.getWhere());
+
+        final List<SqlNode> orderExprList = new ArrayList<>();
+        final List<RelFieldCollation> collationList = new ArrayList<>();
+        gatherOrderExprs(bb, select, select.getOrderList(), orderExprList, collationList);
+        final RelCollation collation = cluster.traitSet().canonize(RelCollations.of(collationList));
+
+        if (validator.isAggregate(select)) {
+            convertAgg(bb, select, orderExprList);
+        } else {
+            convertSelectList(bb, select, orderExprList);
+        }
+
+        if (select.isDistinct()) {
+            distinctify(bb, true);
+        }
+
+        convertOrder(select, bb, collation, orderExprList, select.getOffset(), select.getFetch());
+
+        if (select.hasHints()) {
+            final List<RelHint> hints = SqlUtil.getRelHint(hintStrategies, select.getHints());
+            // Attach the hints to the first Hintable node we found from the root node.
+            bb.setRoot(
+                    bb.root.accept(
+                            new RelShuttleImpl() {
+                                boolean attached = false;
+
+                                @Override
+                                public RelNode visitChild(RelNode parent, int i, RelNode child) {
+                                    if (parent instanceof Hintable && !attached) {
+                                        attached = true;
+                                        return ((Hintable) parent).attachHints(hints);
+                                    } else {
+                                        return super.visitChild(parent, i, child);
+                                    }
+                                }
+                            }),
+                    true);
+        } else {
+            bb.setRoot(bb.root, true);
+        }
+    }
+
+    /**
+     * Having translated 'SELECT ... FROM ... [GROUP BY ...] [HAVING ...]', adds a relational
+     * expression to make the results unique.
+     *
+     * <p>If the SELECT clause contains duplicate expressions, adds {@link LogicalProject}s so that
+     * we are grouping on the minimal set of keys. The performance gain isn't huge, but it is
+     * difficult to detect these duplicate expressions later.
+     *
+     * @param bb Blackboard
+     * @param checkForDupExprs Check for duplicate expressions
+     */
+    private void distinctify(Blackboard bb, boolean checkForDupExprs) {
+        // Look for duplicate expressions in the project.
+        // Say we have 'select x, y, x, z'.
+        // Then dups will be {[2, 0]}
+        // and oldToNew will be {[0, 0], [1, 1], [2, 0], [3, 2]}
+        RelNode rel = bb.root;
+        if (checkForDupExprs && (rel instanceof LogicalProject)) {
+            LogicalProject project = (LogicalProject) rel;
+            final List<RexNode> projectExprs = project.getProjects();
+            final List<Integer> origins = new ArrayList<>();
+            int dupCount = 0;
+            for (int i = 0; i < projectExprs.size(); i++) {
+                int x = projectExprs.indexOf(projectExprs.get(i));
+                if (x >= 0 && x < i) {
+                    origins.add(x);
+                    ++dupCount;
+                } else {
+                    origins.add(i);
+                }
+            }
+            if (dupCount == 0) {
+                distinctify(bb, false);
+                return;
+            }
+
+            final Map<Integer, Integer> squished = new HashMap<>();
+            final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
+            final List<Pair<RexNode, String>> newProjects = new ArrayList<>();
+            for (int i = 0; i < fields.size(); i++) {
+                if (origins.get(i) == i) {
+                    squished.put(i, newProjects.size());
+                    newProjects.add(RexInputRef.of2(i, fields));
+                }
+            }
+            rel =
+                    LogicalProject.create(
+                            rel,
+                            com.google.common.collect.ImmutableList.of(),
+                            Pair.left(newProjects),
+                            Pair.right(newProjects));
+            bb.root = rel;
+            distinctify(bb, false);
+            rel = bb.root;
+
+            // Create the expressions to reverse the mapping.
+            // Project($0, $1, $0, $2).
+            final List<Pair<RexNode, String>> undoProjects = new ArrayList<>();
+            for (int i = 0; i < fields.size(); i++) {
+                final int origin = origins.get(i);
+                RelDataTypeField field = fields.get(i);
+                undoProjects.add(
+                        Pair.of(
+                                new RexInputRef(squished.get(origin), field.getType()),
+                                field.getName()));
+            }
+
+            rel =
+                    LogicalProject.create(
+                            rel,
+                            com.google.common.collect.ImmutableList.of(),
+                            Pair.left(undoProjects),
+                            Pair.right(undoProjects));
+            bb.setRoot(rel, false);
+
+            return;
+        }
+
+        // Usual case: all of the expressions in the SELECT clause are
+        // different.
+        final ImmutableBitSet groupSet = ImmutableBitSet.range(rel.getRowType().getFieldCount());
+        rel =
+                createAggregate(
+                        bb,
+                        groupSet,
+                        com.google.common.collect.ImmutableList.of(groupSet),
+                        com.google.common.collect.ImmutableList.of());
+
+        bb.setRoot(rel, false);
+    }
+
+    /**
+     * Converts a query's ORDER BY clause, if any.
+     *
+     * <p>Ignores the ORDER BY clause if the query is not top-level and FETCH or OFFSET are not
+     * present.
+     *
+     * @param select Query
+     * @param bb Blackboard
+     * @param collation Collation list
+     * @param orderExprList Method populates this list with orderBy expressions not present in
+     *     selectList
+     * @param offset Expression for number of rows to discard before returning first row
+     * @param fetch Expression for number of rows to fetch
+     */
+    protected void convertOrder(
+            SqlSelect select,
+            Blackboard bb,
+            RelCollation collation,
+            List<SqlNode> orderExprList,
+            SqlNode offset,
+            SqlNode fetch) {
+        if (removeSortInSubQuery(bb.top)
+                || select.getOrderList() == null
+                || select.getOrderList().getList().isEmpty()) {
+            assert removeSortInSubQuery(bb.top) || collation.getFieldCollations().isEmpty();
+            if ((offset == null
+                            || (offset instanceof SqlLiteral
+                                    && ((SqlLiteral) offset)
+                                            .bigDecimalValue()
+                                            .equals(BigDecimal.ZERO)))
+                    && fetch == null) {
+                return;
+            }
+        }
+
+        // Create a sorter using the previously constructed collations.
+        bb.setRoot(
+                LogicalSort.create(
+                        bb.root,
+                        collation,
+                        offset == null ? null : convertExpression(offset),
+                        fetch == null ? null : convertExpression(fetch)),
+                false);
+
+        // If extra expressions were added to the project list for sorting,
+        // add another project to remove them. But make the collation empty, because
+        // we can't represent the real collation.
+        //
+        // If it is the top node, use the real collation, but don't trim fields.
+        if (orderExprList.size() > 0 && !bb.top) {
+            final List<RexNode> exprs = new ArrayList<>();
+            final RelDataType rowType = bb.root.getRowType();
+            final int fieldCount = rowType.getFieldCount() - orderExprList.size();
+            for (int i = 0; i < fieldCount; i++) {
+                exprs.add(rexBuilder.makeInputRef(bb.root, i));
+            }
+            bb.setRoot(
+                    LogicalProject.create(
+                            bb.root,
+                            com.google.common.collect.ImmutableList.of(),
+                            exprs,
+                            rowType.getFieldNames().subList(0, fieldCount)),
+                    false);
+        }
+    }
+
+    /**
+     * Returns whether we should remove the sort for the subsequent query conversion.
+     *
+     * @param top Whether the rel to convert is the root of the query
+     */
+    private boolean removeSortInSubQuery(boolean top) {
+        return config.isRemoveSortInSubQuery() && !top;
+    }
+
+    /**
+     * Returns whether a given node contains a {@link SqlInOperator}.
+     *
+     * @param node a RexNode tree
+     */
+    private static boolean containsInOperator(SqlNode node) {
+        try {
+            SqlVisitor<Void> visitor =
+                    new SqlBasicVisitor<Void>() {
+                        public Void visit(SqlCall call) {
+                            if (call.getOperator() instanceof SqlInOperator) {
+                                throw new Util.FoundOne(call);
+                            }
+                            return super.visit(call);
+                        }
+                    };
+            node.accept(visitor);
+            return false;
+        } catch (Util.FoundOne e) {
+            Util.swallow(e, null);
+            return true;
+        }
+    }
+
+    /**
+     * Push down all the NOT logical operators into any IN/NOT IN operators.
+     *
+     * @param scope Scope where {@code sqlNode} occurs
+     * @param sqlNode the root node from which to look for NOT operators
+     * @return the transformed SqlNode representation with NOT pushed down.
+     */
+    private static SqlNode pushDownNotForIn(SqlValidatorScope scope, SqlNode sqlNode) {
+        if (!(sqlNode instanceof SqlCall) || !containsInOperator(sqlNode)) {
+            return sqlNode;
+        }
+        final SqlCall sqlCall = (SqlCall) sqlNode;
+        switch (sqlCall.getKind()) {
+            case AND:
+            case OR:
+                final List<SqlNode> operands = new ArrayList<>();
+                for (SqlNode operand : sqlCall.getOperandList()) {
+                    operands.add(pushDownNotForIn(scope, operand));
+                }
+                final SqlCall newCall =
+                        sqlCall.getOperator().createCall(sqlCall.getParserPosition(), operands);
+                return reg(scope, newCall);
+
+            case NOT:
+                assert sqlCall.operand(0) instanceof SqlCall;
+                final SqlCall call = sqlCall.operand(0);
+                switch (sqlCall.operand(0).getKind()) {
+                    case CASE:
+                        final SqlCase caseNode = (SqlCase) call;
+                        final SqlNodeList thenOperands = new SqlNodeList(SqlParserPos.ZERO);
+
+                        for (SqlNode thenOperand : caseNode.getThenOperands()) {
+                            final SqlCall not =
+                                    SqlStdOperatorTable.NOT.createCall(
+                                            SqlParserPos.ZERO, thenOperand);
+                            thenOperands.add(pushDownNotForIn(scope, reg(scope, not)));
+                        }
+                        SqlNode elseOperand = caseNode.getElseOperand();
+                        if (!SqlUtil.isNull(elseOperand)) {
+                            // "not(unknown)" is "unknown", so no need to simplify
+                            final SqlCall not =
+                                    SqlStdOperatorTable.NOT.createCall(
+                                            SqlParserPos.ZERO, elseOperand);
+                            elseOperand = pushDownNotForIn(scope, reg(scope, not));
+                        }
+
+                        return reg(
+                                scope,
+                                SqlStdOperatorTable.CASE.createCall(
+                                        SqlParserPos.ZERO,
+                                        caseNode.getValueOperand(),
+                                        caseNode.getWhenOperands(),
+                                        thenOperands,
+                                        elseOperand));
+
+                    case AND:
+                        final List<SqlNode> orOperands = new ArrayList<>();
+                        for (SqlNode operand : call.getOperandList()) {
+                            orOperands.add(
+                                    pushDownNotForIn(
+                                            scope,
+                                            reg(
+                                                    scope,
+                                                    SqlStdOperatorTable.NOT.createCall(
+                                                            SqlParserPos.ZERO, operand))));
+                        }
+                        return reg(
+                                scope,
+                                SqlStdOperatorTable.OR.createCall(SqlParserPos.ZERO, orOperands));
+
+                    case OR:
+                        final List<SqlNode> andOperands = new ArrayList<>();
+                        for (SqlNode operand : call.getOperandList()) {
+                            andOperands.add(
+                                    pushDownNotForIn(
+                                            scope,
+                                            reg(
+                                                    scope,
+                                                    SqlStdOperatorTable.NOT.createCall(
+                                                            SqlParserPos.ZERO, operand))));
+                        }
+                        return reg(
+                                scope,
+                                SqlStdOperatorTable.AND.createCall(SqlParserPos.ZERO, andOperands));
+
+                    case NOT:
+                        assert call.operandCount() == 1;
+                        return pushDownNotForIn(scope, call.operand(0));
+
+                    case NOT_IN:
+                        return reg(
+                                scope,
+                                SqlStdOperatorTable.IN.createCall(
+                                        SqlParserPos.ZERO, call.getOperandList()));
+
+                    case IN:
+                        return reg(
+                                scope,
+                                SqlStdOperatorTable.NOT_IN.createCall(
+                                        SqlParserPos.ZERO, call.getOperandList()));
+                }
+        }
+        return sqlNode;
+    }
+
+    /**
+     * Registers with the validator a {@link SqlNode} that has been created during the Sql-to-Rel
+     * process.
+     */
+    private static SqlNode reg(SqlValidatorScope scope, SqlNode e) {
+        scope.getValidator().deriveType(scope, e);
+        return e;
+    }
+
+    /**
+     * Converts a WHERE clause.
+     *
+     * @param bb Blackboard
+     * @param where WHERE clause, may be null
+     */
+    private void convertWhere(final Blackboard bb, final SqlNode where) {
+        if (where == null) {
+            return;
+        }
+        SqlNode newWhere = pushDownNotForIn(bb.scope, where);
+        replaceSubQueries(bb, newWhere, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
+        final RexNode convertedWhere = bb.convertExpression(newWhere);
+        final RexNode convertedWhere2 = RexUtil.removeNullabilityCast(typeFactory, convertedWhere);
+
+        // only allocate filter if the condition is not TRUE
+        if (convertedWhere2.isAlwaysTrue()) {
+            return;
+        }
+
+        final RelFactories.FilterFactory filterFactory = RelFactories.DEFAULT_FILTER_FACTORY;
+        final RelNode filter =
+                filterFactory.createFilter(
+                        bb.root, convertedWhere2, com.google.common.collect.ImmutableSet.of());
+        final RelNode r;
+        final CorrelationUse p = getCorrelationUse(bb, filter);
+        if (p != null) {
+            assert p.r instanceof Filter;
+            Filter f = (Filter) p.r;
+            r =
+                    LogicalFilter.create(
+                            f.getInput(),
+                            f.getCondition(),
+                            com.google.common.collect.ImmutableSet.of(p.id));
+        } else {
+            r = filter;
+        }
+
+        bb.setRoot(r, false);
+    }
+
+    private void replaceSubQueries(
+            final Blackboard bb, final SqlNode expr, RelOptUtil.Logic logic) {
+        findSubQueries(bb, expr, logic, false);
+        for (SubQuery node : bb.subQueryList) {
+            substituteSubQuery(bb, node);
+        }
+    }
+
+    private void substituteSubQuery(Blackboard bb, SubQuery subQuery) {
+        final RexNode expr = subQuery.expr;
+        if (expr != null) {
+            // Already done.
+            return;
+        }
+
+        final SqlBasicCall call;
+        final RelNode rel;
+        final SqlNode query;
+        final RelOptUtil.Exists converted;
+        switch (subQuery.node.getKind()) {
+            case CURSOR:
+                convertCursor(bb, subQuery);
+                return;
+
+            case MULTISET_QUERY_CONSTRUCTOR:
+            case MULTISET_VALUE_CONSTRUCTOR:
+            case ARRAY_QUERY_CONSTRUCTOR:
+                rel =
+                        convertMultisets(
+                                com.google.common.collect.ImmutableList.of(subQuery.node), bb);
+                subQuery.expr = bb.register(rel, JoinRelType.INNER);
+                return;
+
+            case IN:
+            case NOT_IN:
+            case SOME:
+            case ALL:
+                call = (SqlBasicCall) subQuery.node;
+                query = call.operand(1);
+                if (!config.isExpand() && !(query instanceof SqlNodeList)) {
+                    return;
+                }
+                final SqlNode leftKeyNode = call.operand(0);
+
+                final List<RexNode> leftKeys;
+                switch (leftKeyNode.getKind()) {
+                    case ROW:
+                        leftKeys = new ArrayList<>();
+                        for (SqlNode sqlExpr : ((SqlBasicCall) leftKeyNode).getOperandList()) {
+                            leftKeys.add(bb.convertExpression(sqlExpr));
+                        }
+                        break;
+                    default:
+                        leftKeys =
+                                com.google.common.collect.ImmutableList.of(
+                                        bb.convertExpression(leftKeyNode));
+                }
+
+                if (query instanceof SqlNodeList) {
+                    SqlNodeList valueList = (SqlNodeList) query;
+                    if (!containsNullLiteral(valueList)
+                            && valueList.size() < config.getInSubQueryThreshold()) {
+                        // We're under the threshold, so convert to OR.
+                        subQuery.expr =
+                                convertInToOr(
+                                        bb,
+                                        leftKeys,
+                                        valueList,
+                                        (SqlInOperator) call.getOperator());
+                        return;
+                    }
+
+                    // Otherwise, let convertExists translate
+                    // values list into an inline table for the
+                    // reference to Q below.
+                }
+
+                // Project out the search columns from the left side
+
+                // Q1:
+                // "select from emp where emp.deptno in (select col1 from T)"
+                //
+                // is converted to
+                //
+                // "select from
+                //   emp inner join (select distinct col1 from T)) q
+                //   on emp.deptno = q.col1
+                //
+                // Q2:
+                // "select from emp where emp.deptno not in (Q)"
+                //
+                // is converted to
+                //
+                // "select from
+                //   emp left outer join (select distinct col1, TRUE from T) q
+                //   on emp.deptno = q.col1
+                //   where emp.deptno <> null
+                //         and q.indicator <> TRUE"
+                //
+                // Note: Sub-query can be used as SqlUpdate#condition like below:
+                //
+                //   UPDATE emp
+                //   SET empno = 1 WHERE emp.empno IN (
+                //     SELECT emp.empno FROM emp WHERE emp.empno = 2)
+                //
+                // In such case, when converting SqlUpdate#condition, bb.root is null
+                // and it makes no sense to do the sub-query substitution.
+                if (bb.root == null) {
+                    return;
+                }
+                final RelDataType targetRowType =
+                        SqlTypeUtil.promoteToRowType(
+                                typeFactory, validator.getValidatedNodeType(leftKeyNode), null);
+                final boolean notIn = call.getOperator().kind == SqlKind.NOT_IN;
+                converted =
+                        convertExists(
+                                query,
+                                RelOptUtil.SubQueryType.IN,
+                                subQuery.logic,
+                                notIn,
+                                targetRowType);
+                if (converted.indicator) {
+                    // Generate
+                    //    emp CROSS JOIN (SELECT COUNT(*) AS c,
+                    //                       COUNT(deptno) AS ck FROM dept)
+                    final RelDataType longType = typeFactory.createSqlType(SqlTypeName.BIGINT);
+                    final RelNode seek = converted.r.getInput(0); // fragile
+                    final int keyCount = leftKeys.size();
+                    final List<Integer> args = ImmutableIntList.range(0, keyCount);
+                    LogicalAggregate aggregate =
+                            LogicalAggregate.create(
+                                    seek,
+                                    com.google.common.collect.ImmutableList.of(),
+                                    ImmutableBitSet.of(),
+                                    null,
+                                    com.google.common.collect.ImmutableList.of(
+                                            AggregateCall.create(
+                                                    SqlStdOperatorTable.COUNT,
+                                                    false,
+                                                    false,
+                                                    false,
+                                                    com.google.common.collect.ImmutableList.of(),
+                                                    -1,
+                                                    RelCollations.EMPTY,
+                                                    longType,
+                                                    null),
+                                            AggregateCall.create(
+                                                    SqlStdOperatorTable.COUNT,
+                                                    false,
+                                                    false,
+                                                    false,
+                                                    args,
+                                                    -1,
+                                                    RelCollations.EMPTY,
+                                                    longType,
+                                                    null)));
+                    LogicalJoin join =
+                            LogicalJoin.create(
+                                    bb.root,
+                                    aggregate,
+                                    com.google.common.collect.ImmutableList.of(),
+                                    rexBuilder.makeLiteral(true),
+                                    com.google.common.collect.ImmutableSet.of(),
+                                    JoinRelType.INNER);
+                    bb.setRoot(join, false);
+                }
+                final RexNode rex =
+                        bb.register(
+                                converted.r,
+                                converted.outerJoin ? JoinRelType.LEFT : JoinRelType.INNER,
+                                leftKeys);
+
+                RelOptUtil.Logic logic = subQuery.logic;
+                switch (logic) {
+                    case TRUE_FALSE_UNKNOWN:
+                    case UNKNOWN_AS_TRUE:
+                        if (!converted.indicator) {
+                            logic = RelOptUtil.Logic.TRUE_FALSE;
+                        }
+                }
+                subQuery.expr = translateIn(logic, bb.root, rex);
+                if (notIn) {
+                    subQuery.expr = rexBuilder.makeCall(SqlStdOperatorTable.NOT, subQuery.expr);
+                }
+                return;
+
+            case EXISTS:
+                // "select from emp where exists (select a from T)"
+                //
+                // is converted to the following if the sub-query is correlated:
+                //
+                // "select from emp left outer join (select AGG_TRUE() as indicator
+                // from T group by corr_var) q where q.indicator is true"
+                //
+                // If there is no correlation, the expression is replaced with a
+                // boolean indicating whether the sub-query returned 0 or >= 1 row.
+                call = (SqlBasicCall) subQuery.node;
+                query = call.operand(0);
+                if (!config.isExpand()) {
+                    return;
+                }
+                final SqlValidatorScope seekScope =
+                        (query instanceof SqlSelect)
+                                ? validator.getSelectScope((SqlSelect) query)
+                                : null;
+                final Blackboard seekBb = createBlackboard(seekScope, null, false);
+                final RelNode seekRel = convertQueryOrInList(seekBb, query, null);
+                // An EXIST sub-query whose inner child has at least 1 tuple
+                // (e.g. an Aggregate with no grouping columns or non-empty Values
+                // node) should be simplified to a Boolean constant expression.
+                final RelMetadataQuery mq = seekRel.getCluster().getMetadataQuery();
+                final Double minRowCount = mq.getMinRowCount(seekRel);
+                if (minRowCount != null && minRowCount >= 1D) {
+                    subQuery.expr = rexBuilder.makeLiteral(true);
+                    return;
+                }
+                converted =
+                        RelOptUtil.createExistsPlan(
+                                seekRel,
+                                RelOptUtil.SubQueryType.EXISTS,
+                                subQuery.logic,
+                                true,
+                                relBuilder);
+                assert !converted.indicator;
+                if (convertNonCorrelatedSubQuery(subQuery, bb, converted.r, true)) {
+                    return;
+                }
+                subQuery.expr = bb.register(converted.r, JoinRelType.LEFT);
+                return;
+
+            case SCALAR_QUERY:
+                // Convert the sub-query.  If it's non-correlated, convert it
+                // to a constant expression.
+                if (!config.isExpand()) {
+                    return;
+                }
+                call = (SqlBasicCall) subQuery.node;
+                query = call.operand(0);
+                converted =
+                        convertExists(
+                                query, RelOptUtil.SubQueryType.SCALAR, subQuery.logic, true, null);
+                assert !converted.indicator;
+                if (convertNonCorrelatedSubQuery(subQuery, bb, converted.r, false)) {
+                    return;
+                }
+                rel = convertToSingleValueSubq(query, converted.r);
+                subQuery.expr = bb.register(rel, JoinRelType.LEFT);
+                return;
+
+            case SELECT:
+                // This is used when converting multiset queries:
+                //
+                // select * from unnest(select multiset[deptno] from emps);
+                //
+                converted =
+                        convertExists(
+                                subQuery.node,
+                                RelOptUtil.SubQueryType.SCALAR,
+                                subQuery.logic,
+                                true,
+                                null);
+                assert !converted.indicator;
+                subQuery.expr = bb.register(converted.r, JoinRelType.LEFT);
+
+                // This is used when converting window table functions:
+                //
+                // select * from table(tumble(table emps, descriptor(deptno), interval '3' DAY))
+                //
+                bb.cursors.add(converted.r);
+                return;
+
+            default:
+                throw new AssertionError("unexpected kind of sub-query: " + subQuery.node);
+        }
+    }
+
+    private RexNode translateIn(RelOptUtil.Logic logic, RelNode root, final RexNode rex) {
+        switch (logic) {
+            case TRUE:
+                return rexBuilder.makeLiteral(true);
+
+            case TRUE_FALSE:
+            case UNKNOWN_AS_FALSE:
+                assert rex instanceof RexRangeRef;
+                final int fieldCount = rex.getType().getFieldCount();
+                RexNode rexNode = rexBuilder.makeFieldAccess(rex, fieldCount - 1);
+                rexNode = rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, rexNode);
+
+                // Then append the IS NOT NULL(leftKeysForIn).
+                //
+                // RexRangeRef contains the following fields:
+                //   leftKeysForIn,
+                //   rightKeysForIn (the original sub-query select list),
+                //   nullIndicator
+                //
+                // The first two lists contain the same number of fields.
+                final int k = (fieldCount - 1) / 2;
+                for (int i = 0; i < k; i++) {
+                    rexNode =
+                            rexBuilder.makeCall(
+                                    SqlStdOperatorTable.AND,
+                                    rexNode,
+                                    rexBuilder.makeCall(
+                                            SqlStdOperatorTable.IS_NOT_NULL,
+                                            rexBuilder.makeFieldAccess(rex, i)));
+                }
+                return rexNode;
+
+            case TRUE_FALSE_UNKNOWN:
+            case UNKNOWN_AS_TRUE:
+                // select e.deptno,
+                //   case
+                //   when ct.c = 0 then false
+                //   when dt.i is not null then true
+                //   when e.deptno is null then null
+                //   when ct.ck < ct.c then null
+                //   else false
+                //   end
+                // from e
+                // cross join (select count(*) as c, count(deptno) as ck from v) as ct
+                // left join (select distinct deptno, true as i from v) as dt
+                //   on e.deptno = dt.deptno
+                final Join join = (Join) root;
+                final Project left = (Project) join.getLeft();
+                final RelNode leftLeft = ((Join) left.getInput()).getLeft();
+                final int leftLeftCount = leftLeft.getRowType().getFieldCount();
+                final RelDataType longType = typeFactory.createSqlType(SqlTypeName.BIGINT);
+                final RexNode cRef = rexBuilder.makeInputRef(root, leftLeftCount);
+                final RexNode ckRef = rexBuilder.makeInputRef(root, leftLeftCount + 1);
+                final RexNode iRef =
+                        rexBuilder.makeInputRef(root, root.getRowType().getFieldCount() - 1);
+
+                final RexLiteral zero = rexBuilder.makeExactLiteral(BigDecimal.ZERO, longType);
+                final RexLiteral trueLiteral = rexBuilder.makeLiteral(true);
+                final RexLiteral falseLiteral = rexBuilder.makeLiteral(false);
+                final RexNode unknownLiteral = rexBuilder.makeNullLiteral(trueLiteral.getType());
+
+                final com.google.common.collect.ImmutableList.Builder<RexNode> args =
+                        com.google.common.collect.ImmutableList.builder();
+                args.add(
+                        rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, cRef, zero),
+                        falseLiteral,
+                        rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, iRef),
+                        trueLiteral);
+                final JoinInfo joinInfo = join.analyzeCondition();
+                for (int leftKey : joinInfo.leftKeys) {
+                    final RexNode kRef = rexBuilder.makeInputRef(root, leftKey);
+                    args.add(
+                            rexBuilder.makeCall(SqlStdOperatorTable.IS_NULL, kRef), unknownLiteral);
+                }
+                args.add(
+                        rexBuilder.makeCall(SqlStdOperatorTable.LESS_THAN, ckRef, cRef),
+                        unknownLiteral,
+                        falseLiteral);
+
+                return rexBuilder.makeCall(SqlStdOperatorTable.CASE, args.build());
+
+            default:
+                throw new AssertionError(logic);
+        }
+    }
+
+    private static boolean containsNullLiteral(SqlNodeList valueList) {
+        for (SqlNode node : valueList.getList()) {
+            if (node instanceof SqlLiteral) {
+                SqlLiteral lit = (SqlLiteral) node;
+                if (lit.getValue() == null) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Determines if a sub-query is non-correlated and if so, converts it to a constant.
+     *
+     * @param subQuery the call that references the sub-query
+     * @param bb blackboard used to convert the sub-query
+     * @param converted RelNode tree corresponding to the sub-query
+     * @param isExists true if the sub-query is part of an EXISTS expression
+     * @return Whether the sub-query can be converted to a constant
+     */
+    private boolean convertNonCorrelatedSubQuery(
+            SubQuery subQuery, Blackboard bb, RelNode converted, boolean isExists) {
+        SqlCall call = (SqlBasicCall) subQuery.node;
+        if (subQueryConverter.canConvertSubQuery() && isSubQueryNonCorrelated(converted, bb)) {
+            // First check if the sub-query has already been converted
+            // because it's a nested sub-query.  If so, don't re-evaluate
+            // it again.
+            RexNode constExpr = mapConvertedNonCorrSubqs.get(call);
+            if (constExpr == null) {
+                constExpr =
+                        subQueryConverter.convertSubQuery(call, this, isExists, config.isExplain());
+            }
+            if (constExpr != null) {
+                subQuery.expr = constExpr;
+                mapConvertedNonCorrSubqs.put(call, constExpr);
+                return true;
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Converts the RelNode tree for a select statement to a select that produces a single value.
+     *
+     * @param query the query
+     * @param plan the original RelNode tree corresponding to the statement
+     * @return the converted RelNode tree
+     */
+    public RelNode convertToSingleValueSubq(SqlNode query, RelNode plan) {
+        // Check whether query is guaranteed to produce a single value.
+        if (query instanceof SqlSelect) {
+            SqlSelect select = (SqlSelect) query;
+            SqlNodeList selectList = select.getSelectList();
+            SqlNodeList groupList = select.getGroup();
+
+            if ((selectList.size() == 1) && ((groupList == null) || (groupList.size() == 0))) {
+                SqlNode selectExpr = selectList.get(0);
+                if (selectExpr instanceof SqlCall) {
+                    SqlCall selectExprCall = (SqlCall) selectExpr;
+                    if (Util.isSingleValue(selectExprCall)) {
+                        return plan;
+                    }
+                }
+
+                // If there is a limit with 0 or 1,
+                // it is ensured to produce a single value
+                if (select.getFetch() != null && select.getFetch() instanceof SqlNumericLiteral) {
+                    SqlNumericLiteral limitNum = (SqlNumericLiteral) select.getFetch();
+                    if (((BigDecimal) limitNum.getValue()).intValue() < 2) {
+                        return plan;
+                    }
+                }
+            }
+        } else if (query instanceof SqlCall) {
+            // If the query is (values ...),
+            // it is necessary to look into the operands to determine
+            // whether SingleValueAgg is necessary
+            SqlCall exprCall = (SqlCall) query;
+            if (exprCall.getOperator() instanceof SqlValuesOperator
+                    && Util.isSingleValue(exprCall)) {
+                return plan;
+            }
+        }
+
+        // If not, project SingleValueAgg
+        return RelOptUtil.createSingleValueAggRel(cluster, plan);
+    }
+
+    /**
+     * Converts "x IN (1, 2, ...)" to "x=1 OR x=2 OR ...".
+     *
+     * @param leftKeys LHS
+     * @param valuesList RHS
+     * @param op The operator (IN, NOT IN, &gt; SOME, ...)
+     * @return converted expression
+     */
+    private RexNode convertInToOr(
+            final Blackboard bb,
+            final List<RexNode> leftKeys,
+            SqlNodeList valuesList,
+            SqlInOperator op) {
+        final List<RexNode> comparisons = new ArrayList<>();
+        for (SqlNode rightVals : valuesList) {
+            RexNode rexComparison;
+            final SqlOperator comparisonOp;
+            if (op instanceof SqlQuantifyOperator) {
+                comparisonOp =
+                        RelOptUtil.op(
+                                ((SqlQuantifyOperator) op).comparisonKind,
+                                SqlStdOperatorTable.EQUALS);
+            } else {
+                comparisonOp = SqlStdOperatorTable.EQUALS;
+            }
+            if (leftKeys.size() == 1) {
+                rexComparison =
+                        rexBuilder.makeCall(
+                                comparisonOp,
+                                leftKeys.get(0),
+                                ensureSqlType(
+                                        leftKeys.get(0).getType(),
+                                        bb.convertExpression(rightVals)));
+            } else {
+                assert rightVals instanceof SqlCall;
+                final SqlBasicCall call = (SqlBasicCall) rightVals;
+                assert (call.getOperator() instanceof SqlRowOperator)
+                        && call.operandCount() == leftKeys.size();
+                rexComparison =
+                        RexUtil.composeConjunction(
+                                rexBuilder,
+                                Util.transform(
+                                        Pair.zip(leftKeys, call.getOperandList()),
+                                        pair ->
+                                                rexBuilder.makeCall(
+                                                        comparisonOp,
+                                                        pair.left,
+                                                        ensureSqlType(
+                                                                pair.left.getType(),
+                                                                bb.convertExpression(
+                                                                        pair.right)))));
+            }
+            comparisons.add(rexComparison);
+        }
+
+        switch (op.kind) {
+            case ALL:
+                return RexUtil.composeConjunction(rexBuilder, comparisons, true);
+            case NOT_IN:
+                return rexBuilder.makeCall(
+                        SqlStdOperatorTable.NOT,
+                        RexUtil.composeDisjunction(rexBuilder, comparisons, true));
+            case IN:
+            case SOME:
+                return RexUtil.composeDisjunction(rexBuilder, comparisons, true);
+            default:
+                throw new AssertionError();
+        }
+    }
+
+    /**
+     * Ensures that an expression has a given {@link SqlTypeName}, applying a cast if necessary. If
+     * the expression already has the right type family, returns the expression unchanged.
+     */
+    private RexNode ensureSqlType(RelDataType type, RexNode node) {
+        if (type.getSqlTypeName() == node.getType().getSqlTypeName()
+                || (type.getSqlTypeName() == SqlTypeName.VARCHAR
+                        && node.getType().getSqlTypeName() == SqlTypeName.CHAR)) {
+            return node;
+        }
+        return rexBuilder.ensureType(type, node, true);
+    }
+
+    /**
+     * Gets the list size threshold under which {@link #convertInToOr} is used. Lists of this size
+     * or greater will instead be converted to use a join against an inline table ({@link
+     * LogicalValues}) rather than a predicate. A threshold of 0 forces usage of an inline table in
+     * all cases; a threshold of Integer.MAX_VALUE forces usage of OR in all cases
+     *
+     * @return threshold, default {@link #DEFAULT_IN_SUB_QUERY_THRESHOLD}
+     */
+    @Deprecated // to be removed before 2.0
+    protected int getInSubqueryThreshold() {
+        return config.getInSubQueryThreshold();
+    }
+
+    /**
+     * Converts an EXISTS or IN predicate into a join. For EXISTS, the sub-query produces an
+     * indicator variable, and the result is a relational expression which outer joins that
+     * indicator to the original query. After performing the outer join, the condition will be TRUE
+     * if the EXISTS condition holds, NULL otherwise.
+     *
+     * @param seek A query, for example 'select * from emp' or 'values (1,2,3)' or '('Foo', 34)'.
+     * @param subQueryType Whether sub-query is IN, EXISTS or scalar
+     * @param logic Whether the answer needs to be in full 3-valued logic (TRUE, FALSE, UNKNOWN)
+     *     will be required, or whether we can accept an approximation (say representing UNKNOWN as
+     *     FALSE)
+     * @param notIn Whether the operation is NOT IN
+     * @return join expression
+     */
+    private RelOptUtil.Exists convertExists(
+            SqlNode seek,
+            RelOptUtil.SubQueryType subQueryType,
+            RelOptUtil.Logic logic,
+            boolean notIn,
+            RelDataType targetDataType) {
+        final SqlValidatorScope seekScope =
+                (seek instanceof SqlSelect) ? validator.getSelectScope((SqlSelect) seek) : null;
+        final Blackboard seekBb = createBlackboard(seekScope, null, false);
+        RelNode seekRel = convertQueryOrInList(seekBb, seek, targetDataType);
+
+        return RelOptUtil.createExistsPlan(seekRel, subQueryType, logic, notIn, relBuilder);
+    }
+
+    private RelNode convertQueryOrInList(Blackboard bb, SqlNode seek, RelDataType targetRowType) {
+        // NOTE: Once we start accepting single-row queries as row constructors,
+        // there will be an ambiguity here for a case like X IN ((SELECT Y FROM
+        // Z)).  The SQL standard resolves the ambiguity by saying that a lone
+        // select should be interpreted as a table expression, not a row
+        // expression.  The semantic difference is that a table expression can
+        // return multiple rows.
+        if (seek instanceof SqlNodeList) {
+            return convertRowValues(bb, seek, ((SqlNodeList) seek).getList(), false, targetRowType);
+        } else {
+            return convertQueryRecursive(seek, false, null).project();
+        }
+    }
+
+    private RelNode convertRowValues(
+            Blackboard bb,
+            SqlNode rowList,
+            Collection<SqlNode> rows,
+            boolean allowLiteralsOnly,
+            RelDataType targetRowType) {
+        // NOTE jvs 30-Apr-2006: We combine all rows consisting entirely of
+        // literals into a single LogicalValues; this gives the optimizer a smaller
+        // input tree.  For everything else (computed expressions, row
+        // sub-queries), we union each row in as a projection on top of a
+        // LogicalOneRow.
+
+        final com.google.common.collect.ImmutableList.Builder<
+                        com.google.common.collect.ImmutableList<RexLiteral>>
+                tupleList = com.google.common.collect.ImmutableList.builder();
+        final RelDataType rowType;
+        if (targetRowType != null) {
+            rowType = targetRowType;
+        } else {
+            rowType =
+                    SqlTypeUtil.promoteToRowType(
+                            typeFactory, validator.getValidatedNodeType(rowList), null);
+        }
+
+        final List<RelNode> unionInputs = new ArrayList<>();
+        for (SqlNode node : rows) {
+            SqlBasicCall call;
+            if (isRowConstructor(node)) {
+                call = (SqlBasicCall) node;
+                com.google.common.collect.ImmutableList.Builder<RexLiteral> tuple =
+                        com.google.common.collect.ImmutableList.builder();
+                for (Ord<SqlNode> operand : Ord.zip(call.operands)) {
+                    RexLiteral rexLiteral =
+                            convertLiteralInValuesList(operand.e, bb, rowType, operand.i);
+                    if ((rexLiteral == null) && allowLiteralsOnly) {
+                        return null;
+                    }
+                    if ((rexLiteral == null) || !config.isCreateValuesRel()) {
+                        // fallback to convertRowConstructor
+                        tuple = null;
+                        break;
+                    }
+                    tuple.add(rexLiteral);
+                }
+                if (tuple != null) {
+                    tupleList.add(tuple.build());
+                    continue;
+                }
+            } else {
+                RexLiteral rexLiteral = convertLiteralInValuesList(node, bb, rowType, 0);
+                if ((rexLiteral != null) && config.isCreateValuesRel()) {
+                    tupleList.add(com.google.common.collect.ImmutableList.of(rexLiteral));
+                    continue;
+                } else {
+                    if ((rexLiteral == null) && allowLiteralsOnly) {
+                        return null;
+                    }
+                }
+
+                // convert "1" to "row(1)"
+                call = (SqlBasicCall) SqlStdOperatorTable.ROW.createCall(SqlParserPos.ZERO, node);
+            }
+            unionInputs.add(convertRowConstructor(bb, call));
+        }
+        LogicalValues values = LogicalValues.create(cluster, rowType, tupleList.build());
+        RelNode resultRel;
+        if (unionInputs.isEmpty()) {
+            resultRel = values;
+        } else {
+            if (!values.getTuples().isEmpty()) {
+                unionInputs.add(values);
+            }
+            resultRel = LogicalUnion.create(unionInputs, true);
+        }
+        leaves.put(resultRel, resultRel.getRowType().getFieldCount());
+        return resultRel;
+    }
+
+    private RexLiteral convertLiteralInValuesList(
+            SqlNode sqlNode, Blackboard bb, RelDataType rowType, int iField) {
+        if (!(sqlNode instanceof SqlLiteral)) {
+            return null;
+        }
+        RelDataTypeField field = rowType.getFieldList().get(iField);
+        RelDataType type = field.getType();
+        if (type.isStruct()) {
+            // null literals for weird stuff like UDT's need
+            // special handling during type flattening, so
+            // don't use LogicalValues for those
+            return null;
+        }
+
+        RexNode literalExpr = exprConverter.convertLiteral(bb, (SqlLiteral) sqlNode);
+
+        if (!(literalExpr instanceof RexLiteral)) {
+            assert literalExpr.isA(SqlKind.CAST);
+            RexNode child = ((RexCall) literalExpr).getOperands().get(0);
+            assert RexLiteral.isNullLiteral(child);
+
+            // NOTE jvs 22-Nov-2006:  we preserve type info
+            // in LogicalValues digest, so it's OK to lose it here
+            return (RexLiteral) child;
+        }
+
+        RexLiteral literal = (RexLiteral) literalExpr;
+
+        Comparable value = literal.getValue();
+
+        if (SqlTypeUtil.isExactNumeric(type) && SqlTypeUtil.hasScale(type)) {
+            BigDecimal roundedValue =
+                    NumberUtil.rescaleBigDecimal((BigDecimal) value, type.getScale());
+            return rexBuilder.makeExactLiteral(roundedValue, type);
+        }
+
+        if ((value instanceof NlsString) && (type.getSqlTypeName() == SqlTypeName.CHAR)) {
+            // pad fixed character type
+            NlsString unpadded = (NlsString) value;
+            return rexBuilder.makeCharLiteral(
+                    new NlsString(
+                            Spaces.padRight(unpadded.getValue(), type.getPrecision()),
+                            unpadded.getCharsetName(),
+                            unpadded.getCollation()));
+        }
+        return literal;
+    }
+
+    private boolean isRowConstructor(SqlNode node) {
+        if (!(node.getKind() == SqlKind.ROW)) {
+            return false;
+        }
+        SqlCall call = (SqlCall) node;
+        return call.getOperator().getName().equalsIgnoreCase("row");
+    }
+
+    /**
+     * Builds a list of all <code>IN</code> or <code>EXISTS</code> operators inside SQL parse tree.
+     * Does not traverse inside queries.
+     *
+     * @param bb blackboard
+     * @param node the SQL parse tree
+     * @param logic Whether the answer needs to be in full 3-valued logic (TRUE, FALSE, UNKNOWN)
+     *     will be required, or whether we can accept an approximation (say representing UNKNOWN as
+     *     FALSE)
+     * @param registerOnlyScalarSubQueries if set to true and the parse tree corresponds to a
+     *     variation of a select node, only register it if it's a scalar sub-query
+     */
+    private void findSubQueries(
+            Blackboard bb,
+            SqlNode node,
+            RelOptUtil.Logic logic,
+            boolean registerOnlyScalarSubQueries) {
+        final SqlKind kind = node.getKind();
+        switch (kind) {
+            case EXISTS:
+            case SELECT:
+            case MULTISET_QUERY_CONSTRUCTOR:
+            case MULTISET_VALUE_CONSTRUCTOR:
+            case ARRAY_QUERY_CONSTRUCTOR:
+            case CURSOR:
+            case SCALAR_QUERY:
+                if (!registerOnlyScalarSubQueries || (kind == SqlKind.SCALAR_QUERY)) {
+                    bb.registerSubQuery(node, RelOptUtil.Logic.TRUE_FALSE);
+                }
+                return;
+            case IN:
+                break;
+            case NOT_IN:
+            case NOT:
+                logic = logic.negate();
+                break;
+        }
+        if (node instanceof SqlCall) {
+            switch (kind) {
+                    // Do no change logic for AND, IN and NOT IN expressions;
+                    // but do change logic for OR, NOT and others;
+                    // EXISTS was handled already.
+                case AND:
+                case IN:
+                case NOT_IN:
+                    break;
+                default:
+                    logic = RelOptUtil.Logic.TRUE_FALSE_UNKNOWN;
+                    break;
+            }
+            for (SqlNode operand : ((SqlCall) node).getOperandList()) {
+                if (operand != null) {
+                    // In the case of an IN expression, locate scalar
+                    // sub-queries so we can convert them to constants
+                    findSubQueries(
+                            bb,
+                            operand,
+                            logic,
+                            kind == SqlKind.IN
+                                    || kind == SqlKind.NOT_IN
+                                    || kind == SqlKind.SOME
+                                    || kind == SqlKind.ALL
+                                    || registerOnlyScalarSubQueries);
+                }
+            }
+        } else if (node instanceof SqlNodeList) {
+            for (SqlNode child : (SqlNodeList) node) {
+                findSubQueries(
+                        bb,
+                        child,
+                        logic,
+                        kind == SqlKind.IN
+                                || kind == SqlKind.NOT_IN
+                                || kind == SqlKind.SOME
+                                || kind == SqlKind.ALL
+                                || registerOnlyScalarSubQueries);
+            }
+        }
+
+        // Now that we've located any scalar sub-queries inside the IN
+        // expression, register the IN expression itself.  We need to
+        // register the scalar sub-queries first so they can be converted
+        // before the IN expression is converted.
+        switch (kind) {
+            case IN:
+            case NOT_IN:
+            case SOME:
+            case ALL:
+                switch (logic) {
+                    case TRUE_FALSE_UNKNOWN:
+                        RelDataType type = validator.getValidatedNodeTypeIfKnown(node);
+                        if (type == null) {
+                            // The node might not be validated if we still don't know type of the
+                            // node.
+                            // Therefore return directly.
+                            return;
+                        } else {
+                            break;
+                        }
+                        // fall through
+                    case UNKNOWN_AS_FALSE:
+                        logic = RelOptUtil.Logic.TRUE;
+                }
+                bb.registerSubQuery(node, logic);
+                break;
+        }
+    }
+
+    /**
+     * Converts an expression from {@link SqlNode} to {@link RexNode} format.
+     *
+     * @param node Expression to translate
+     * @return Converted expression
+     */
+    public RexNode convertExpression(SqlNode node) {
+        Map<String, RelDataType> nameToTypeMap = Collections.emptyMap();
+        final ParameterScope scope =
+                new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap);
+        final Blackboard bb = createBlackboard(scope, null, false);
+        return bb.convertExpression(node);
+    }
+
+    /**
+     * Converts an expression from {@link SqlNode} to {@link RexNode} format, mapping identifier
+     * references to predefined expressions.
+     *
+     * @param node Expression to translate
+     * @param nameToNodeMap map from String to {@link RexNode}; when an {@link SqlIdentifier} is
+     *     encountered, it is used as a key and translated to the corresponding value from this map
+     * @return Converted expression
+     */
+    public RexNode convertExpression(SqlNode node, Map<String, RexNode> nameToNodeMap) {
+        final Map<String, RelDataType> nameToTypeMap = new HashMap<>();
+        for (Map.Entry<String, RexNode> entry : nameToNodeMap.entrySet()) {
+            nameToTypeMap.put(entry.getKey(), entry.getValue().getType());
+        }
+        final ParameterScope scope =
+                new ParameterScope((SqlValidatorImpl) validator, nameToTypeMap);
+        final Blackboard bb = createBlackboard(scope, nameToNodeMap, false);
+        return bb.convertExpression(node);
+    }
+
+    /**
+     * Converts a non-standard expression.
+     *
+     * <p>This method is an extension-point that derived classes can override. If this method
+     * returns a null result, the normal expression translation process will proceed. The default
+     * implementation always returns null.
+     *
+     * @param node Expression
+     * @param bb Blackboard
+     * @return null to proceed with the usual expression translation process
+     */
+    protected RexNode convertExtendedExpression(SqlNode node, Blackboard bb) {
+        return null;
+    }
+
+    private RexNode convertOver(Blackboard bb, SqlNode node) {
+        SqlCall call = (SqlCall) node;
+        SqlCall aggCall = call.operand(0);
+        boolean ignoreNulls = false;
+        switch (aggCall.getKind()) {
+            case IGNORE_NULLS:
+                ignoreNulls = true;
+                // fall through
+            case RESPECT_NULLS:
+                aggCall = aggCall.operand(0);
+        }
+
+        SqlNode windowOrRef = call.operand(1);
+        final SqlWindow window = validator.resolveWindow(windowOrRef, bb.scope);
+
+        SqlNode sqlLowerBound = window.getLowerBound();
+        SqlNode sqlUpperBound = window.getUpperBound();
+        boolean rows = window.isRows();
+        SqlNodeList orderList = window.getOrderList();
+
+        if (!aggCall.getOperator().allowsFraming()) {
+            // If the operator does not allow framing, bracketing is implicitly
+            // everything up to the current row.
+            sqlLowerBound = SqlWindow.createUnboundedPreceding(SqlParserPos.ZERO);
+            sqlUpperBound = SqlWindow.createCurrentRow(SqlParserPos.ZERO);
+            if (aggCall.getKind() == SqlKind.ROW_NUMBER) {
+                // ROW_NUMBER() expects specific kind of framing.
+                rows = true;
+            }
+        } else if (orderList.size() == 0) {
+            // Without ORDER BY, there must be no bracketing.
+            sqlLowerBound = SqlWindow.createUnboundedPreceding(SqlParserPos.ZERO);
+            sqlUpperBound = SqlWindow.createUnboundedFollowing(SqlParserPos.ZERO);
+        } else if (sqlLowerBound == null && sqlUpperBound == null) {
+            sqlLowerBound = SqlWindow.createUnboundedPreceding(SqlParserPos.ZERO);
+            sqlUpperBound = SqlWindow.createCurrentRow(SqlParserPos.ZERO);
+        } else if (sqlUpperBound == null) {
+            sqlUpperBound = SqlWindow.createCurrentRow(SqlParserPos.ZERO);
+        } else if (sqlLowerBound == null) {
+            sqlLowerBound = SqlWindow.createCurrentRow(SqlParserPos.ZERO);
+        }
+        final SqlNodeList partitionList = window.getPartitionList();
+        final com.google.common.collect.ImmutableList.Builder<RexNode> partitionKeys =
+                com.google.common.collect.ImmutableList.builder();
+        for (SqlNode partition : partitionList) {
+            partitionKeys.add(bb.convertExpression(partition));
+        }
+        final RexNode lowerBound = bb.convertExpression(sqlLowerBound);
+        final RexNode upperBound = bb.convertExpression(sqlUpperBound);
+        if (orderList.size() == 0 && !rows) {
+            // A logical range requires an ORDER BY clause. Use the implicit
+            // ordering of this relation. There must be one, otherwise it would
+            // have failed validation.
+            orderList = bb.scope.getOrderList();
+            if (orderList == null) {
+                throw new AssertionError("Relation should have sort key for implicit ORDER BY");
+            }
+        }
+
+        final com.google.common.collect.ImmutableList.Builder<RexFieldCollation> orderKeys =
+                com.google.common.collect.ImmutableList.builder();
+        for (SqlNode order : orderList) {
+            orderKeys.add(
+                    bb.convertSortExpression(
+                            order,
+                            RelFieldCollation.Direction.ASCENDING,
+                            RelFieldCollation.NullDirection.UNSPECIFIED));
+        }
+
+        try {
+            com.google.common.base.Preconditions.checkArgument(
+                    bb.window == null, "already in window agg mode");
+            bb.window = window;
+            RexNode rexAgg = exprConverter.convertCall(bb, aggCall);
+            rexAgg = rexBuilder.ensureType(validator.getValidatedNodeType(call), rexAgg, false);
+
+            // Walk over the tree and apply 'over' to all agg functions. This is
+            // necessary because the returned expression is not necessarily a call
+            // to an agg function. For example, AVG(x) becomes SUM(x) / COUNT(x).
+
+            final SqlLiteral q = aggCall.getFunctionQuantifier();
+            final boolean isDistinct = q != null && q.getValue() == SqlSelectKeyword.DISTINCT;
+
+            final RexShuttle visitor =
+                    new HistogramShuttle(
+                            partitionKeys.build(),
+                            orderKeys.build(),
+                            RexWindowBounds.create(sqlLowerBound, lowerBound),
+                            RexWindowBounds.create(sqlUpperBound, upperBound),
+                            rows,
+                            window.isAllowPartial(),
+                            isDistinct,
+                            ignoreNulls);
+            return rexAgg.accept(visitor);
+        } finally {
+            bb.window = null;
+        }
+    }
+
+    protected void convertFrom(Blackboard bb, SqlNode from) {
+        convertFrom(bb, from, Collections.emptyList());
+    }
+
+    /**
+     * Converts a FROM clause into a relational expression.
+     *
+     * @param bb Scope within which to resolve identifiers
+     * @param from FROM clause of a query. Examples include:
+     *     <ul>
+     *       <li>a single table ("SALES.EMP"),
+     *       <li>an aliased table ("EMP AS E"),
+     *       <li>a list of tables ("EMP, DEPT"),
+     *       <li>an ANSI Join expression ("EMP JOIN DEPT ON EMP.DEPTNO = DEPT.DEPTNO"),
+     *       <li>a VALUES clause ("VALUES ('Fred', 20)"),
+     *       <li>a query ("(SELECT * FROM EMP WHERE GENDER = 'F')"),
+     *       <li>or any combination of the above.
+     *     </ul>
+     *
+     * @param fieldNames Field aliases, usually come from AS clause
+     */
+    protected void convertFrom(Blackboard bb, SqlNode from, List<String> fieldNames) {
+        if (from == null) {
+            bb.setRoot(LogicalValues.createOneRow(cluster), false);
+            return;
+        }
+
+        final SqlCall call;
+        final SqlNode[] operands;
+        switch (from.getKind()) {
+            case AS:
+                call = (SqlCall) from;
+                SqlNode firstOperand = call.operand(0);
+                final List<String> fieldNameList = new ArrayList<>();
+                if (call.operandCount() > 2) {
+                    for (SqlNode node : Util.skip(call.getOperandList(), 2)) {
+                        fieldNameList.add(((SqlIdentifier) node).getSimple());
+                    }
+                }
+                convertFrom(bb, firstOperand, fieldNameList);
+                return;
+
+            case MATCH_RECOGNIZE:
+                convertMatchRecognize(bb, (SqlMatchRecognize) from);
+                return;
+
+            case PIVOT:
+                convertPivot(bb, (SqlPivot) from);
+                return;
+
+            case WITH_ITEM:
+                convertFrom(bb, ((SqlWithItem) from).query);
+                return;
+
+            case WITH:
+                convertFrom(bb, ((SqlWith) from).body);
+                return;
+
+            case TABLESAMPLE:
+                operands = ((SqlBasicCall) from).getOperands();
+                SqlSampleSpec sampleSpec = SqlLiteral.sampleValue(operands[1]);
+                if (sampleSpec instanceof SqlSampleSpec.SqlSubstitutionSampleSpec) {
+                    String sampleName =
+                            ((SqlSampleSpec.SqlSubstitutionSampleSpec) sampleSpec).getName();
+                    datasetStack.push(sampleName);
+                    convertFrom(bb, operands[0]);
+                    datasetStack.pop();
+                } else if (sampleSpec instanceof SqlSampleSpec.SqlTableSampleSpec) {
+                    SqlSampleSpec.SqlTableSampleSpec tableSampleSpec =
+                            (SqlSampleSpec.SqlTableSampleSpec) sampleSpec;
+                    convertFrom(bb, operands[0]);
+                    RelOptSamplingParameters params =
+                            new RelOptSamplingParameters(
+                                    tableSampleSpec.isBernoulli(),
+                                    tableSampleSpec.getSamplePercentage(),
+                                    tableSampleSpec.isRepeatable(),
+                                    tableSampleSpec.getRepeatableSeed());
+                    bb.setRoot(new Sample(cluster, bb.root, params), false);
+                } else {
+                    throw new AssertionError("unknown TABLESAMPLE type: " + sampleSpec);
+                }
+                return;
+
+            case TABLE_REF:
+                call = (SqlCall) from;
+                convertIdentifier(bb, call.operand(0), null, call.operand(1));
+                return;
+
+            case IDENTIFIER:
+                convertIdentifier(bb, (SqlIdentifier) from, null, null);
+                return;
+
+            case EXTEND:
+                call = (SqlCall) from;
+                final SqlNode operand0 = call.getOperandList().get(0);
+                final SqlIdentifier id =
+                        operand0.getKind() == SqlKind.TABLE_REF
+                                ? ((SqlCall) operand0).operand(0)
+                                : (SqlIdentifier) operand0;
+                SqlNodeList extendedColumns = (SqlNodeList) call.getOperandList().get(1);
+                convertIdentifier(bb, id, extendedColumns, null);
+                return;
+
+            case SNAPSHOT:
+                convertTemporalTable(bb, (SqlCall) from);
+                return;
+
+            case JOIN:
+                convertJoin(bb, (SqlJoin) from);
+                return;
+
+            case SELECT:
+            case INTERSECT:
+            case EXCEPT:
+            case UNION:
+                final RelNode rel = convertQueryRecursive(from, false, null).project();
+                bb.setRoot(rel, true);
+                return;
+
+            case VALUES:
+                convertValuesImpl(bb, (SqlCall) from, null);
+                if (fieldNames.size() > 0) {
+                    bb.setRoot(relBuilder.push(bb.root).rename(fieldNames).build(), true);
+                }
+                return;
+
+            case UNNEST:
+                convertUnnest(bb, (SqlCall) from, fieldNames);
+                return;
+
+            case COLLECTION_TABLE:
+                call = (SqlCall) from;
+
+                // Dig out real call; TABLE() wrapper is just syntactic.
+                assert call.getOperandList().size() == 1;
+                final SqlCall call2 = call.operand(0);
+                convertCollectionTable(bb, call2);
+                return;
+
+            default:
+                throw new AssertionError("not a join operator " + from);
+        }
+    }
+
+    private void convertUnnest(Blackboard bb, SqlCall call, List<String> fieldNames) {
+        final List<SqlNode> nodes = call.getOperandList();
+        final SqlUnnestOperator operator = (SqlUnnestOperator) call.getOperator();
+        for (SqlNode node : nodes) {
+            replaceSubQueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+        }
+        final List<RexNode> exprs = new ArrayList<>();
+        for (Ord<SqlNode> node : Ord.zip(nodes)) {
+            exprs.add(
+                    relBuilder.alias(
+                            bb.convertExpression(node.e), validator.deriveAlias(node.e, node.i)));
+        }
+        RelNode child = (null != bb.root) ? bb.root : LogicalValues.createOneRow(cluster);
+        RelNode uncollect;
+        if (validator.config().sqlConformance().allowAliasUnnestItems()) {
+            uncollect =
+                    relBuilder
+                            .push(child)
+                            .project(exprs)
+                            .uncollect(fieldNames, operator.withOrdinality)
+                            .build();
+        } else {
+            // REVIEW danny 2020-04-26: should we unify the normal field aliases and the item
+            // aliases ?
+            uncollect =
+                    relBuilder
+                            .push(child)
+                            .project(exprs)
+                            .uncollect(Collections.emptyList(), operator.withOrdinality)
+                            .rename(fieldNames)
+                            .build();
+        }
+        bb.setRoot(uncollect, true);
+    }
+
+    protected void convertMatchRecognize(Blackboard bb, SqlMatchRecognize matchRecognize) {
+        final SqlValidatorNamespace ns = validator.getNamespace(matchRecognize);
+        final SqlValidatorScope scope = validator.getMatchRecognizeScope(matchRecognize);
+
+        final Blackboard matchBb = createBlackboard(scope, null, false);
+        final RelDataType rowType = ns.getRowType();
+        // convert inner query, could be a table name or a derived table
+        SqlNode expr = matchRecognize.getTableRef();
+        convertFrom(matchBb, expr);
+        final RelNode input = matchBb.root;
+
+        // PARTITION BY
+        final SqlNodeList partitionList = matchRecognize.getPartitionList();
+        final ImmutableBitSet.Builder partitionKeys = ImmutableBitSet.builder();
+        for (SqlNode partition : partitionList) {
+            RexNode e = matchBb.convertExpression(partition);
+            partitionKeys.set(((RexInputRef) e).getIndex());
+        }
+
+        // ORDER BY
+        final SqlNodeList orderList = matchRecognize.getOrderList();
+        final List<RelFieldCollation> orderKeys = new ArrayList<>();
+        for (SqlNode order : orderList) {
+            final RelFieldCollation.Direction direction;
+            switch (order.getKind()) {
+                case DESCENDING:
+                    direction = RelFieldCollation.Direction.DESCENDING;
+                    order = ((SqlCall) order).operand(0);
+                    break;
+                case NULLS_FIRST:
+                case NULLS_LAST:
+                    throw new AssertionError();
+                default:
+                    direction = RelFieldCollation.Direction.ASCENDING;
+                    break;
+            }
+            final RelFieldCollation.NullDirection nullDirection =
+                    validator.config().defaultNullCollation().last(desc(direction))
+                            ? RelFieldCollation.NullDirection.LAST
+                            : RelFieldCollation.NullDirection.FIRST;
+            RexNode e = matchBb.convertExpression(order);
+            orderKeys.add(
+                    new RelFieldCollation(((RexInputRef) e).getIndex(), direction, nullDirection));
+        }
+        final RelCollation orders = cluster.traitSet().canonize(RelCollations.of(orderKeys));
+
+        // convert pattern
+        final Set<String> patternVarsSet = new HashSet<>();
+        SqlNode pattern = matchRecognize.getPattern();
+        final SqlBasicVisitor<RexNode> patternVarVisitor =
+                new SqlBasicVisitor<RexNode>() {
+                    @Override
+                    public RexNode visit(SqlCall call) {
+                        List<SqlNode> operands = call.getOperandList();
+                        List<RexNode> newOperands = new ArrayList<>();
+                        for (SqlNode node : operands) {
+                            newOperands.add(node.accept(this));
+                        }
+                        return rexBuilder.makeCall(
+                                validator.getUnknownType(), call.getOperator(), newOperands);
+                    }
+
+                    @Override
+                    public RexNode visit(SqlIdentifier id) {
+                        assert id.isSimple();
+                        patternVarsSet.add(id.getSimple());
+                        return rexBuilder.makeLiteral(id.getSimple());
+                    }
+
+                    @Override
+                    public RexNode visit(SqlLiteral literal) {
+                        if (literal instanceof SqlNumericLiteral) {
+                            return rexBuilder.makeExactLiteral(
+                                    BigDecimal.valueOf(literal.intValue(true)));
+                        } else {
+                            return rexBuilder.makeLiteral(literal.booleanValue());
+                        }
+                    }
+                };
+        final RexNode patternNode = pattern.accept(patternVarVisitor);
+
+        SqlLiteral interval = matchRecognize.getInterval();
+        RexNode intervalNode = null;
+        if (interval != null) {
+            intervalNode = matchBb.convertLiteral(interval);
+        }
+
+        // convert subset
+        final SqlNodeList subsets = matchRecognize.getSubsetList();
+        final Map<String, TreeSet<String>> subsetMap = new HashMap<>();
+        for (SqlNode node : subsets) {
+            List<SqlNode> operands = ((SqlCall) node).getOperandList();
+            SqlIdentifier left = (SqlIdentifier) operands.get(0);
+            patternVarsSet.add(left.getSimple());
+            SqlNodeList rights = (SqlNodeList) operands.get(1);
+            final TreeSet<String> list = new TreeSet<>();
+            for (SqlNode right : rights) {
+                assert right instanceof SqlIdentifier;
+                list.add(((SqlIdentifier) right).getSimple());
+            }
+            subsetMap.put(left.getSimple(), list);
+        }
+
+        SqlNode afterMatch = matchRecognize.getAfter();
+        if (afterMatch == null) {
+            afterMatch = SqlMatchRecognize.AfterOption.SKIP_TO_NEXT_ROW.symbol(SqlParserPos.ZERO);
+        }
+
+        final RexNode after;
+        if (afterMatch instanceof SqlCall) {
+            List<SqlNode> operands = ((SqlCall) afterMatch).getOperandList();
+            SqlOperator operator = ((SqlCall) afterMatch).getOperator();
+            assert operands.size() == 1;
+            SqlIdentifier id = (SqlIdentifier) operands.get(0);
+            assert patternVarsSet.contains(id.getSimple())
+                    : id.getSimple() + " not defined in pattern";
+            RexNode rex = rexBuilder.makeLiteral(id.getSimple());
+            after =
+                    rexBuilder.makeCall(
+                            validator.getUnknownType(),
+                            operator,
+                            com.google.common.collect.ImmutableList.of(rex));
+        } else {
+            after = matchBb.convertExpression(afterMatch);
+        }
+
+        matchBb.setPatternVarRef(true);
+
+        // convert measures
+        final com.google.common.collect.ImmutableMap.Builder<String, RexNode> measureNodes =
+                com.google.common.collect.ImmutableMap.builder();
+        for (SqlNode measure : matchRecognize.getMeasureList()) {
+            List<SqlNode> operands = ((SqlCall) measure).getOperandList();
+            String alias = ((SqlIdentifier) operands.get(1)).getSimple();
+            RexNode rex = matchBb.convertExpression(operands.get(0));
+            measureNodes.put(alias, rex);
+        }
+
+        // convert definitions
+        final com.google.common.collect.ImmutableMap.Builder<String, RexNode> definitionNodes =
+                com.google.common.collect.ImmutableMap.builder();
+        for (SqlNode def : matchRecognize.getPatternDefList()) {
+            replaceSubQueries(matchBb, def, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
+            List<SqlNode> operands = ((SqlCall) def).getOperandList();
+            String alias = ((SqlIdentifier) operands.get(1)).getSimple();
+            RexNode rex = matchBb.convertExpression(operands.get(0));
+            definitionNodes.put(alias, rex);
+        }
+
+        final SqlLiteral rowsPerMatch = matchRecognize.getRowsPerMatch();
+        final boolean allRows =
+                rowsPerMatch != null
+                        && rowsPerMatch.getValue() == SqlMatchRecognize.RowsPerMatchOption.ALL_ROWS;
+
+        matchBb.setPatternVarRef(false);
+
+        final RelFactories.MatchFactory factory = RelFactories.DEFAULT_MATCH_FACTORY;
+        final RelNode rel =
+                factory.createMatch(
+                        input,
+                        patternNode,
+                        rowType,
+                        matchRecognize.getStrictStart().booleanValue(),
+                        matchRecognize.getStrictEnd().booleanValue(),
+                        definitionNodes.build(),
+                        measureNodes.build(),
+                        after,
+                        subsetMap,
+                        allRows,
+                        partitionKeys.build(),
+                        orders,
+                        intervalNode);
+        bb.setRoot(rel, false);
+    }
+
+    protected void convertPivot(Blackboard bb, SqlPivot pivot) {
+        final SqlValidatorNamespace ns = validator.getNamespace(pivot);
+        final SqlValidatorScope scope = validator.getJoinScope(pivot);
+
+        final Blackboard pivotBb = createBlackboard(scope, null, false);
+
+        // Convert input
+        convertFrom(pivotBb, pivot.query);
+        final RelNode input = pivotBb.root;
+
+        final RelDataType inputRowType = input.getRowType();
+        relBuilder.push(input);
+
+        // Gather fields.
+        final AggConverter aggConverter = new AggConverter(pivotBb, (AggregatingSelectScope) null);
+        final Set<String> usedColumnNames = pivot.usedColumnNames();
+
+        // 1. Gather group keys.
+        inputRowType.getFieldList().stream()
+                .filter(field -> !usedColumnNames.contains(field.getName()))
+                .forEach(
+                        field ->
+                                aggConverter.addGroupExpr(
+                                        new SqlIdentifier(field.getName(), SqlParserPos.ZERO)));
+
+        // 2. Gather axes.
+        pivot.axisList.forEach(aggConverter::addGroupExpr);
+
+        // 3. Gather columns used as arguments to aggregate functions.
+        pivotBb.agg = aggConverter;
+        final List<String> aggAliasList = new ArrayList<>();
+        assert aggConverter.aggCalls.size() == 0;
+        pivot.forEachAgg(
+                (alias, call) -> {
+                    call.accept(aggConverter);
+                    aggAliasList.add(alias);
+                    assert aggConverter.aggCalls.size() == aggAliasList.size();
+                });
+        pivotBb.agg = null;
+
+        // Project the fields that we will need.
+        relBuilder.project(
+                Pair.left(aggConverter.getPreExprs()), Pair.right(aggConverter.getPreExprs()));
+
+        // Build expressions.
+
+        // 1. Build group key
+        final RelBuilder.GroupKey groupKey =
+                relBuilder.groupKey(
+                        inputRowType.getFieldList().stream()
+                                .filter(field -> !usedColumnNames.contains(field.getName()))
+                                .map(
+                                        field ->
+                                                aggConverter.addGroupExpr(
+                                                        new SqlIdentifier(
+                                                                field.getName(),
+                                                                SqlParserPos.ZERO)))
+                                .collect(ImmutableBitSet.toImmutableBitSet()));
+
+        // 2. Build axes, for example
+        // FOR (axis1, axis2 ...) IN ...
+        final List<RexNode> axes = new ArrayList<>();
+        for (SqlNode axis : pivot.axisList) {
+            axes.add(relBuilder.field(aggConverter.addGroupExpr(axis)));
+        }
+
+        // 3. Build aggregate expressions, for example
+        // PIVOT (sum(a) AS alias1, min(b) AS alias2, ... FOR ... IN ...)
+        final List<RelBuilder.AggCall> aggCalls = new ArrayList<>();
+        Pair.forEach(
+                aggAliasList,
+                aggConverter.aggCalls,
+                (alias, aggregateCall) ->
+                        aggCalls.add(relBuilder.aggregateCall(aggregateCall).as(alias)));
+
+        // 4. Build values, for example
+        // IN ((v11, v12, ...) AS label1, (v21, v22, ...) AS label2, ...)
+        final com.google.common.collect.ImmutableList.Builder<Pair<String, List<RexNode>>>
+                valueList = com.google.common.collect.ImmutableList.builder();
+        pivot.forEachNameValues(
+                (alias, nodeList) ->
+                        valueList.add(
+                                Pair.of(
+                                        alias,
+                                        nodeList.getList().stream()
+                                                .map(bb::convertExpression)
+                                                .collect(Util.toImmutableList()))));
+
+        final RelNode rel = relBuilder.pivot(groupKey, aggCalls, axes, valueList.build()).build();
+        bb.setRoot(rel, true);
+    }
+
+    private void convertIdentifier(
+            Blackboard bb, SqlIdentifier id, SqlNodeList extendedColumns, SqlNodeList tableHints) {
+        final SqlValidatorNamespace fromNamespace = validator.getNamespace(id).resolve();
+        if (fromNamespace.getNode() != null) {
+            convertFrom(bb, fromNamespace.getNode());
+            return;
+        }
+        final String datasetName = datasetStack.isEmpty() ? null : datasetStack.peek();
+        final boolean[] usedDataset = {false};
+        RelOptTable table =
+                SqlValidatorUtil.getRelOptTable(
+                        fromNamespace, catalogReader, datasetName, usedDataset);
+        if (extendedColumns != null && extendedColumns.size() > 0) {
+            assert table != null;
+            final SqlValidatorTable validatorTable = table.unwrap(SqlValidatorTable.class);
+            final List<RelDataTypeField> extendedFields =
+                    SqlValidatorUtil.getExtendedColumns(validator, validatorTable, extendedColumns);
+            table = table.extend(extendedFields);
+        }
+        final RelNode tableRel;
+        // Review Danny 2020-01-13: hacky to construct a new table scan
+        // in order to apply the hint strategies.
+        final List<RelHint> hints =
+                hintStrategies.apply(
+                        SqlUtil.getRelHint(hintStrategies, tableHints),
+                        LogicalTableScan.create(
+                                cluster, table, com.google.common.collect.ImmutableList.of()));
+        tableRel = toRel(table, hints);
+        bb.setRoot(tableRel, true);
+        if (usedDataset[0]) {
+            bb.setDataset(datasetName);
+        }
+    }
+
+    protected void convertCollectionTable(Blackboard bb, SqlCall call) {
+        final SqlOperator operator = call.getOperator();
+        if (operator == SqlStdOperatorTable.TABLESAMPLE) {
+            final String sampleName = SqlLiteral.unchain(call.operand(0)).getValueAs(String.class);
+            datasetStack.push(sampleName);
+            SqlCall cursorCall = call.operand(1);
+            SqlNode query = cursorCall.operand(0);
+            RelNode converted = convertQuery(query, false, false).rel;
+            bb.setRoot(converted, false);
+            datasetStack.pop();
+            return;
+        }
+        replaceSubQueries(bb, call, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+        // Expand table macro if possible. It's more efficient than
+        // LogicalTableFunctionScan.
+        final SqlCallBinding callBinding =
+                new SqlCallBinding(bb.scope.getValidator(), bb.scope, call);
+        if (operator instanceof SqlUserDefinedTableMacro) {
+            final SqlUserDefinedTableMacro udf = (SqlUserDefinedTableMacro) operator;
+            final TranslatableTable table = udf.getTable(callBinding);
+            final RelDataType rowType = table.getRowType(typeFactory);
+            RelOptTable relOptTable =
+                    RelOptTableImpl.create(null, rowType, table, udf.getNameAsId().names);
+            RelNode converted = toRel(relOptTable, com.google.common.collect.ImmutableList.of());
+            bb.setRoot(converted, true);
+            return;
+        }
+
+        Type elementType;
+        if (operator instanceof SqlUserDefinedTableFunction) {
+            SqlUserDefinedTableFunction udtf = (SqlUserDefinedTableFunction) operator;
+            elementType = udtf.getElementType(callBinding);
+        } else {
+            elementType = null;
+        }
+
+        RexNode rexCall = bb.convertExpression(call);
+        final List<RelNode> inputs = bb.retrieveCursors();
+        Set<RelColumnMapping> columnMappings = getColumnMappings(operator);
+        LogicalTableFunctionScan callRel =
+                LogicalTableFunctionScan.create(
+                        cluster,
+                        inputs,
+                        rexCall,
+                        elementType,
+                        validator.getValidatedNodeType(call),
+                        columnMappings);
+        bb.setRoot(callRel, true);
+        afterTableFunction(bb, call, callRel);
+    }
+
+    protected void afterTableFunction(
+            Blackboard bb, SqlCall call, LogicalTableFunctionScan callRel) {}
+
+    private void convertTemporalTable(Blackboard bb, SqlCall call) {
+        final SqlSnapshot snapshot = (SqlSnapshot) call;
+        final RexNode period = bb.convertExpression(snapshot.getPeriod());
+
+        // convert inner query, could be a table name or a derived table
+        SqlNode expr = snapshot.getTableRef();
+        convertFrom(bb, expr);
+
+        final RelNode snapshotRel = relBuilder.push(bb.root).snapshot(period).build();
+
+        bb.setRoot(snapshotRel, false);
+    }
+
+    private Set<RelColumnMapping> getColumnMappings(SqlOperator op) {
+        SqlReturnTypeInference rti = op.getReturnTypeInference();
+        if (rti == null) {
+            return null;
+        }
+        if (rti instanceof TableFunctionReturnTypeInference) {
+            TableFunctionReturnTypeInference tfrti = (TableFunctionReturnTypeInference) rti;
+            return tfrti.getColumnMappings();
+        } else {
+            return null;
+        }
+    }
+
+    /**
+     * Shuttle that replace outer {@link RexInputRef} with {@link RexFieldAccess}, and adjust {@code
+     * offset} to each inner {@link RexInputRef} in the lateral join condition.
+     */
+    private static class RexAccessShuttle extends RexShuttle {
+        private final RexBuilder builder;
+        private final RexCorrelVariable rexCorrel;
+        private final BitSet varCols = new BitSet();
+
+        RexAccessShuttle(RexBuilder builder, RexCorrelVariable rexCorrel) {
+            this.builder = builder;
+            this.rexCorrel = rexCorrel;
+        }
+
+        @Override
+        public RexNode visitInputRef(RexInputRef input) {
+            int i = input.getIndex() - rexCorrel.getType().getFieldCount();
+            if (i < 0) {
+                varCols.set(input.getIndex());
+                return builder.makeFieldAccess(rexCorrel, input.getIndex());
+            }
+            return builder.makeInputRef(input.getType(), i);
+        }
+    }
+
+    protected RelNode createJoin(
+            Blackboard bb,
+            RelNode leftRel,
+            RelNode rightRel,
+            RexNode joinCond,
+            JoinRelType joinType) {
+        assert joinCond != null;
+
+        final CorrelationUse p = getCorrelationUse(bb, rightRel);
+        if (p != null) {
+            RelNode innerRel = p.r;
+            ImmutableBitSet requiredCols = p.requiredColumns;
+
+            if (!joinCond.isAlwaysTrue()) {
+                final RelFactories.FilterFactory factory = RelFactories.DEFAULT_FILTER_FACTORY;
+                final RexCorrelVariable rexCorrel =
+                        (RexCorrelVariable) rexBuilder.makeCorrel(leftRel.getRowType(), p.id);
+                final RexAccessShuttle shuttle = new RexAccessShuttle(rexBuilder, rexCorrel);
+
+                // Replace outer RexInputRef with RexFieldAccess,
+                // and push lateral join predicate into inner child
+                final RexNode newCond = joinCond.accept(shuttle);
+                innerRel =
+                        factory.createFilter(
+                                p.r, newCond, com.google.common.collect.ImmutableSet.of());
+                requiredCols = ImmutableBitSet.fromBitSet(shuttle.varCols).union(p.requiredColumns);
+            }
+
+            return LogicalCorrelate.create(leftRel, innerRel, p.id, requiredCols, joinType);
+        }
+
+        final RelNode node =
+                relBuilder.push(leftRel).push(rightRel).join(joinType, joinCond).build();
+
+        // If join conditions are pushed down, update the leaves.
+        if (node instanceof Project) {
+            final Join newJoin = (Join) node.getInputs().get(0);
+            if (leaves.containsKey(leftRel)) {
+                leaves.put(newJoin.getLeft(), leaves.get(leftRel));
+            }
+            if (leaves.containsKey(rightRel)) {
+                leaves.put(newJoin.getRight(), leaves.get(rightRel));
+            }
+        }
+        return node;
+    }
+
+    private CorrelationUse getCorrelationUse(Blackboard bb, final RelNode r0) {
+        final Set<CorrelationId> correlatedVariables = RelOptUtil.getVariablesUsed(r0);
+        if (correlatedVariables.isEmpty()) {
+            return null;
+        }
+        final ImmutableBitSet.Builder requiredColumns = ImmutableBitSet.builder();
+        final List<CorrelationId> correlNames = new ArrayList<>();
+
+        // All correlations must refer the same namespace since correlation
+        // produces exactly one correlation source.
+        // The same source might be referenced by different variables since
+        // DeferredLookups are not de-duplicated at create time.
+        SqlValidatorNamespace prevNs = null;
+
+        for (CorrelationId correlName : correlatedVariables) {
+            DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
+            RexFieldAccess fieldAccess = lookup.getFieldAccess(correlName);
+            String originalRelName = lookup.getOriginalRelName();
+            String originalFieldName = fieldAccess.getField().getName();
+
+            final SqlNameMatcher nameMatcher = bb.getValidator().getCatalogReader().nameMatcher();
+            final SqlValidatorScope.ResolvedImpl resolved = new SqlValidatorScope.ResolvedImpl();
+            lookup.bb.scope.resolve(
+                    com.google.common.collect.ImmutableList.of(originalRelName),
+                    nameMatcher,
+                    false,
+                    resolved);
+            assert resolved.count() == 1;
+            final SqlValidatorScope.Resolve resolve = resolved.only();
+            final SqlValidatorNamespace foundNs = resolve.namespace;
+            final RelDataType rowType = resolve.rowType();
+            final int childNamespaceIndex = resolve.path.steps().get(0).i;
+            final SqlValidatorScope ancestorScope = resolve.scope;
+            boolean correlInCurrentScope = bb.scope.isWithin(ancestorScope);
+
+            if (!correlInCurrentScope) {
+                continue;
+            }
+
+            if (prevNs == null) {
+                prevNs = foundNs;
+            } else {
+                assert prevNs == foundNs
+                        : "All correlation variables should resolve"
+                                + " to the same namespace."
+                                + " Prev ns="
+                                + prevNs
+                                + ", new ns="
+                                + foundNs;
+            }
+
+            int namespaceOffset = 0;
+            if (childNamespaceIndex > 0) {
+                // If not the first child, need to figure out the width
+                // of output types from all the preceding namespaces
+                assert ancestorScope instanceof ListScope;
+                List<SqlValidatorNamespace> children = ((ListScope) ancestorScope).getChildren();
+
+                for (int i = 0; i < childNamespaceIndex; i++) {
+                    SqlValidatorNamespace child = children.get(i);
+                    namespaceOffset += child.getRowType().getFieldCount();
+                }
+            }
+
+            RexFieldAccess topLevelFieldAccess = fieldAccess;
+            while (topLevelFieldAccess.getReferenceExpr() instanceof RexFieldAccess) {
+                topLevelFieldAccess = (RexFieldAccess) topLevelFieldAccess.getReferenceExpr();
+            }
+            final RelDataTypeField field =
+                    rowType.getFieldList()
+                            .get(topLevelFieldAccess.getField().getIndex() - namespaceOffset);
+            int pos = namespaceOffset + field.getIndex();
+
+            assert field.getType() == topLevelFieldAccess.getField().getType();
+
+            assert pos != -1;
+
+            if (bb.mapRootRelToFieldProjection.containsKey(bb.root)) {
+                // bb.root is an aggregate and only projects group by
+                // keys.
+                Map<Integer, Integer> exprProjection = bb.mapRootRelToFieldProjection.get(bb.root);
+
+                // sub-query can reference group by keys projected from
+                // the root of the outer relation.
+                if (exprProjection.containsKey(pos)) {
+                    pos = exprProjection.get(pos);
+                } else {
+                    // correl not grouped
+                    throw new AssertionError(
+                            "Identifier '"
+                                    + originalRelName
+                                    + "."
+                                    + originalFieldName
+                                    + "' is not a group expr");
+                }
+            }
+
+            requiredColumns.set(pos);
+            correlNames.add(correlName);
+        }
+
+        if (correlNames.isEmpty()) {
+            // None of the correlating variables originated in this scope.
+            return null;
+        }
+
+        RelNode r = r0;
+        if (correlNames.size() > 1) {
+            // The same table was referenced more than once.
+            // So we deduplicate.
+            r =
+                    DeduplicateCorrelateVariables.go(
+                            rexBuilder, correlNames.get(0), Util.skip(correlNames), r0);
+            // Add new node to leaves.
+            leaves.put(r, r.getRowType().getFieldCount());
+        }
+        return new CorrelationUse(correlNames.get(0), requiredColumns.build(), r);
+    }
+
+    /**
+     * Determines whether a sub-query is non-correlated. Note that a non-correlated sub-query can
+     * contain correlated references, provided those references do not reference select statements
+     * that are parents of the sub-query.
+     *
+     * @param subq the sub-query
+     * @param bb blackboard used while converting the sub-query, i.e., the blackboard of the parent
+     *     query of this sub-query
+     * @return true if the sub-query is non-correlated
+     */
+    private boolean isSubQueryNonCorrelated(RelNode subq, Blackboard bb) {
+        Set<CorrelationId> correlatedVariables = RelOptUtil.getVariablesUsed(subq);
+        for (CorrelationId correlName : correlatedVariables) {
+            DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
+            String originalRelName = lookup.getOriginalRelName();
+
+            final SqlNameMatcher nameMatcher =
+                    lookup.bb.scope.getValidator().getCatalogReader().nameMatcher();
+            final SqlValidatorScope.ResolvedImpl resolved = new SqlValidatorScope.ResolvedImpl();
+            lookup.bb.scope.resolve(
+                    com.google.common.collect.ImmutableList.of(originalRelName),
+                    nameMatcher,
+                    false,
+                    resolved);
+
+            SqlValidatorScope ancestorScope = resolved.only().scope;
+
+            // If the correlated reference is in a scope that's "above" the
+            // sub-query, then this is a correlated sub-query.
+            SqlValidatorScope parentScope = bb.scope;
+            do {
+                if (ancestorScope == parentScope) {
+                    return false;
+                }
+                if (parentScope instanceof DelegatingScope) {
+                    parentScope = ((DelegatingScope) parentScope).getParent();
+                } else {
+                    break;
+                }
+            } while (parentScope != null);
+        }
+        return true;
+    }
+
+    /**
+     * Returns a list of fields to be prefixed to each relational expression.
+     *
+     * @return List of system fields
+     */
+    protected List<RelDataTypeField> getSystemFields() {
+        return Collections.emptyList();
+    }
+
+    private void convertJoin(Blackboard bb, SqlJoin join) {
+        final SqlValidatorScope scope = validator.getJoinScope(join);
+        final Blackboard fromBlackboard = createBlackboard(scope, null, false);
+        SqlNode left = join.getLeft();
+        SqlNode right = join.getRight();
+        final SqlValidatorScope leftScope =
+                Util.first(validator.getJoinScope(left), ((DelegatingScope) bb.scope).getParent());
+        final Blackboard leftBlackboard = createBlackboard(leftScope, null, false);
+        final SqlValidatorScope rightScope =
+                Util.first(validator.getJoinScope(right), ((DelegatingScope) bb.scope).getParent());
+        final Blackboard rightBlackboard = createBlackboard(rightScope, null, false);
+        convertFrom(leftBlackboard, left);
+        final RelNode leftRel = leftBlackboard.root;
+        convertFrom(rightBlackboard, right);
+        final RelNode tempRightRel = rightBlackboard.root;
+
+        final JoinConditionType conditionType = join.getConditionType();
+        final RexNode condition;
+        final RelNode rightRel;
+        if (join.isNatural()) {
+            condition =
+                    convertNaturalCondition(
+                            validator.getNamespace(left), validator.getNamespace(right));
+            rightRel = tempRightRel;
+        } else {
+            switch (conditionType) {
+                case NONE:
+                    condition = rexBuilder.makeLiteral(true);
+                    rightRel = tempRightRel;
+                    break;
+                case USING:
+                    condition =
+                            convertUsingCondition(
+                                    join,
+                                    validator.getNamespace(left),
+                                    validator.getNamespace(right));
+                    rightRel = tempRightRel;
+                    break;
+                case ON:
+                    Pair<RexNode, RelNode> conditionAndRightNode =
+                            convertOnCondition(fromBlackboard, join, leftRel, tempRightRel);
+                    condition = conditionAndRightNode.left;
+                    rightRel = conditionAndRightNode.right;
+                    break;
+                default:
+                    throw Util.unexpected(conditionType);
+            }
+        }
+        final RelNode joinRel =
+                createJoin(
+                        fromBlackboard,
+                        leftRel,
+                        rightRel,
+                        condition,
+                        convertJoinType(join.getJoinType()));
+        bb.setRoot(joinRel, false);
+    }
+
+    private RexNode convertNaturalCondition(
+            SqlValidatorNamespace leftNamespace, SqlValidatorNamespace rightNamespace) {
+        final List<String> columnList =
+                SqlValidatorUtil.deriveNaturalJoinColumnList(
+                        catalogReader.nameMatcher(),
+                        leftNamespace.getRowType(),
+                        rightNamespace.getRowType());
+        return convertUsing(leftNamespace, rightNamespace, columnList);
+    }
+
+    private RexNode convertUsingCondition(
+            SqlJoin join,
+            SqlValidatorNamespace leftNamespace,
+            SqlValidatorNamespace rightNamespace) {
+        SqlNode condition = join.getCondition();
+
+        final SqlNodeList list = (SqlNodeList) condition;
+        final List<String> nameList = new ArrayList<>();
+        for (SqlNode columnName : list) {
+            final SqlIdentifier id = (SqlIdentifier) columnName;
+            String name = id.getSimple();
+            nameList.add(name);
+        }
+        return convertUsing(leftNamespace, rightNamespace, nameList);
+    }
+
+    /**
+     * This currently does not expand correlated full outer joins correctly. Replaying on the right
+     * side to correctly support left joins multiplicities.
+     *
+     * <blockquote>
+     *
+     * <pre>
+     *   SELECT *
+     *   FROM t1
+     *   LEFT JOIN t2 ON
+     *    EXIST(SELECT t3.c3 WHERE t1.c1 = t3.c1 AND t2.c2 = t3.c2)
+     *    AND NOT (t2.t2 = 2)
+     * </pre>
+     *
+     * </blockquote>
+     *
+     * <p>Given the de-correlated query produces:
+     *
+     * <blockquote>
+     *
+     * <pre>
+     *  t1.c1 | t2.c2
+     *  ------+------
+     *    1   |  1
+     *    1   |  2
+     * </pre>
+     *
+     * </blockquote>
+     *
+     * <p>If correlated query was replayed on the left side, then an extra rows would be emitted for
+     * every {code t1.c1 = 1}, where it failed to join to right side due to {code NOT(t2.t2 = 2)}.
+     * However, if the query is joined on the right, side multiplicity is maintained.
+     */
+    private Pair<RexNode, RelNode> convertOnCondition(
+            Blackboard bb, SqlJoin join, RelNode leftRel, RelNode rightRel) {
+        SqlNode condition = join.getCondition();
+
+        bb.setRoot(com.google.common.collect.ImmutableList.of(leftRel, rightRel));
+        replaceSubQueries(bb, condition, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
+        final RelNode newRightRel =
+                bb.root == null || bb.registered.size() == 0 ? rightRel : bb.reRegister(rightRel);
+        bb.setRoot(com.google.common.collect.ImmutableList.of(leftRel, newRightRel));
+        RexNode conditionExp = bb.convertExpression(condition);
+        return Pair.of(conditionExp, newRightRel);
+    }
+
+    /**
+     * Returns an expression for matching columns of a USING clause or inferred from NATURAL JOIN.
+     * "a JOIN b USING (x, y)" becomes "a.x = b.x AND a.y = b.y". Returns null if the column list is
+     * empty.
+     *
+     * @param leftNamespace Namespace of left input to join
+     * @param rightNamespace Namespace of right input to join
+     * @param nameList List of column names to join on
+     * @return Expression to match columns from name list, or true if name list is empty
+     */
+    private @Nonnull RexNode convertUsing(
+            SqlValidatorNamespace leftNamespace,
+            SqlValidatorNamespace rightNamespace,
+            List<String> nameList) {
+        final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
+        final List<RexNode> list = new ArrayList<>();
+        for (String name : nameList) {
+            List<RexNode> operands = new ArrayList<>();
+            int offset = 0;
+            for (SqlValidatorNamespace n :
+                    com.google.common.collect.ImmutableList.of(leftNamespace, rightNamespace)) {
+                final RelDataType rowType = n.getRowType();
+                final RelDataTypeField field = nameMatcher.field(rowType, name);
+                operands.add(rexBuilder.makeInputRef(field.getType(), offset + field.getIndex()));
+                offset += rowType.getFieldList().size();
+            }
+            list.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, operands));
+        }
+        return RexUtil.composeConjunction(rexBuilder, list);
+    }
+
+    private static JoinRelType convertJoinType(JoinType joinType) {
+        switch (joinType) {
+            case COMMA:
+            case INNER:
+            case CROSS:
+                return JoinRelType.INNER;
+            case FULL:
+                return JoinRelType.FULL;
+            case LEFT:
+                return JoinRelType.LEFT;
+            case RIGHT:
+                return JoinRelType.RIGHT;
+            default:
+                throw Util.unexpected(joinType);
+        }
+    }
+
+    /**
+     * Converts the SELECT, GROUP BY and HAVING clauses of an aggregate query.
+     *
+     * <p>This method extracts SELECT, GROUP BY and HAVING clauses, and creates an {@link
+     * AggConverter}, then delegates to {@link #createAggImpl}. Derived class may override this
+     * method to change any of those clauses or specify a different {@link AggConverter}.
+     *
+     * @param bb Scope within which to resolve identifiers
+     * @param select Query
+     * @param orderExprList Additional expressions needed to implement ORDER BY
+     */
+    protected void convertAgg(Blackboard bb, SqlSelect select, List<SqlNode> orderExprList) {
+        assert bb.root != null : "precondition: child != null";
+        SqlNodeList groupList = select.getGroup();
+        SqlNodeList selectList = select.getSelectList();
+        SqlNode having = select.getHaving();
+
+        final AggConverter aggConverter = new AggConverter(bb, select);
+        createAggImpl(bb, aggConverter, selectList, groupList, having, orderExprList);
+    }
+
+    protected final void createAggImpl(
+            Blackboard bb,
+            final AggConverter aggConverter,
+            SqlNodeList selectList,
+            SqlNodeList groupList,
+            SqlNode having,
+            List<SqlNode> orderExprList) {
+        // Find aggregate functions in SELECT and HAVING clause
+        final AggregateFinder aggregateFinder = new AggregateFinder();
+        selectList.accept(aggregateFinder);
+        if (having != null) {
+            having.accept(aggregateFinder);
+        }
+
+        // first replace the sub-queries inside the aggregates
+        // because they will provide input rows to the aggregates.
+        replaceSubQueries(bb, aggregateFinder.list, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+        // also replace sub-queries inside filters in the aggregates
+        replaceSubQueries(bb, aggregateFinder.filterList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+        // also replace sub-queries inside ordering spec in the aggregates
+        replaceSubQueries(bb, aggregateFinder.orderList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+        // If group-by clause is missing, pretend that it has zero elements.
+        if (groupList == null) {
+            groupList = SqlNodeList.EMPTY;
+        }
+
+        replaceSubQueries(bb, groupList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+        // register the group exprs
+
+        // build a map to remember the projections from the top scope to the
+        // output of the current root.
+        //
+        // Calcite allows expressions, not just column references in
+        // group by list. This is not SQL 2003 compliant, but hey.
+
+        final AggregatingSelectScope scope = aggConverter.aggregatingSelectScope;
+        final AggregatingSelectScope.Resolved r = scope.resolved.get();
+        for (SqlNode groupExpr : r.groupExprList) {
+            aggConverter.addGroupExpr(groupExpr);
+        }
+
+        final RexNode havingExpr;
+        final List<Pair<RexNode, String>> projects = new ArrayList<>();
+
+        try {
+            com.google.common.base.Preconditions.checkArgument(
+                    bb.agg == null, "already in agg mode");
+            bb.agg = aggConverter;
+
+            // convert the select and having expressions, so that the
+            // agg converter knows which aggregations are required
+
+            selectList.accept(aggConverter);
+            // Assert we don't have dangling items left in the stack
+            assert !aggConverter.inOver;
+            for (SqlNode expr : orderExprList) {
+                expr.accept(aggConverter);
+                assert !aggConverter.inOver;
+            }
+            if (having != null) {
+                having.accept(aggConverter);
+                assert !aggConverter.inOver;
+            }
+
+            // compute inputs to the aggregator
+            List<Pair<RexNode, String>> preExprs = aggConverter.getPreExprs();
+
+            if (preExprs.size() == 0) {
+                // Special case for COUNT(*), where we can end up with no inputs
+                // at all.  The rest of the system doesn't like 0-tuples, so we
+                // select a dummy constant here.
+                final RexNode zero = rexBuilder.makeExactLiteral(BigDecimal.ZERO);
+                preExprs = com.google.common.collect.ImmutableList.of(Pair.of(zero, null));
+            }
+
+            final RelNode inputRel = bb.root;
+
+            // Project the expressions required by agg and having.
+            bb.setRoot(
+                    relBuilder
+                            .push(inputRel)
+                            .projectNamed(Pair.left(preExprs), Pair.right(preExprs), false)
+                            .build(),
+                    false);
+            bb.mapRootRelToFieldProjection.put(bb.root, r.groupExprProjection);
+
+            // REVIEW jvs 31-Oct-2007:  doesn't the declaration of
+            // monotonicity here assume sort-based aggregation at
+            // the physical level?
+
+            // Tell bb which of group columns are sorted.
+            bb.columnMonotonicities.clear();
+            for (SqlNode groupItem : groupList) {
+                bb.columnMonotonicities.add(bb.scope.getMonotonicity(groupItem));
+            }
+
+            // Add the aggregator
+            bb.setRoot(
+                    createAggregate(
+                            bb, r.groupSet, r.groupSets.asList(), aggConverter.getAggCalls()),
+                    false);
+            bb.mapRootRelToFieldProjection.put(bb.root, r.groupExprProjection);
+
+            // Replace sub-queries in having here and modify having to use
+            // the replaced expressions
+            if (having != null) {
+                SqlNode newHaving = pushDownNotForIn(bb.scope, having);
+                replaceSubQueries(bb, newHaving, RelOptUtil.Logic.UNKNOWN_AS_FALSE);
+                havingExpr = bb.convertExpression(newHaving);
+            } else {
+                havingExpr = relBuilder.literal(true);
+            }
+
+            // Now convert the other sub-queries in the select list.
+            // This needs to be done separately from the sub-query inside
+            // any aggregate in the select list, and after the aggregate rel
+            // is allocated.
+            replaceSubQueries(bb, selectList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+            // Now sub-queries in the entire select list have been converted.
+            // Convert the select expressions to get the final list to be
+            // projected.
+            int k = 0;
+
+            // For select expressions, use the field names previously assigned
+            // by the validator. If we derive afresh, we might generate names
+            // like "EXPR$2" that don't match the names generated by the
+            // validator. This is especially the case when there are system
+            // fields; system fields appear in the relnode's rowtype but do not
+            // (yet) appear in the validator type.
+            final SelectScope selectScope = SqlValidatorUtil.getEnclosingSelectScope(bb.scope);
+            assert selectScope != null;
+            final SqlValidatorNamespace selectNamespace =
+                    validator.getNamespace(selectScope.getNode());
+            final List<String> names = selectNamespace.getRowType().getFieldNames();
+            int sysFieldCount = selectList.size() - names.size();
+            for (SqlNode expr : selectList) {
+                projects.add(
+                        Pair.of(
+                                bb.convertExpression(expr),
+                                k < sysFieldCount
+                                        ? validator.deriveAlias(expr, k++)
+                                        : names.get(k++ - sysFieldCount)));
+            }
+
+            for (SqlNode expr : orderExprList) {
+                projects.add(Pair.of(bb.convertExpression(expr), validator.deriveAlias(expr, k++)));
+            }
+        } finally {
+            bb.agg = null;
+        }
+
+        // implement HAVING (we have already checked that it is non-trivial)
+        relBuilder.push(bb.root);
+        if (havingExpr != null) {
+            relBuilder.filter(havingExpr);
+        }
+
+        // implement the SELECT list
+        relBuilder.project(Pair.left(projects), Pair.right(projects)).rename(Pair.right(projects));
+        bb.setRoot(relBuilder.build(), false);
+
+        // Tell bb which of group columns are sorted.
+        bb.columnMonotonicities.clear();
+        for (SqlNode selectItem : selectList) {
+            bb.columnMonotonicities.add(bb.scope.getMonotonicity(selectItem));
+        }
+    }
+
+    /**
+     * Creates an Aggregate.
+     *
+     * <p>In case the aggregate rel changes the order in which it projects fields, the <code>
+     * groupExprProjection</code> parameter is provided, and the implementation of this method may
+     * modify it.
+     *
+     * <p>The <code>sortedCount</code> parameter is the number of expressions known to be monotonic.
+     * These expressions must be on the leading edge of the grouping keys. The default
+     * implementation of this method ignores this parameter.
+     *
+     * @param bb Blackboard
+     * @param groupSet Bit set of ordinals of grouping columns
+     * @param groupSets Grouping sets
+     * @param aggCalls Array of calls to aggregate functions
+     * @return LogicalAggregate
+     */
+    protected RelNode createAggregate(
+            Blackboard bb,
+            ImmutableBitSet groupSet,
+            com.google.common.collect.ImmutableList<ImmutableBitSet> groupSets,
+            List<AggregateCall> aggCalls) {
+        relBuilder.push(bb.root);
+        final RelBuilder.GroupKey groupKey =
+                relBuilder.groupKey(groupSet, (Iterable<ImmutableBitSet>) groupSets);
+        return relBuilder.aggregate(groupKey, aggCalls).build();
+    }
+
+    public RexDynamicParam convertDynamicParam(final SqlDynamicParam dynamicParam) {
+        // REVIEW jvs 8-Jan-2005:  dynamic params may be encountered out of
+        // order.  Should probably cross-check with the count from the parser
+        // at the end and make sure they all got filled in.  Why doesn't List
+        // have a resize() method?!?  Make this a utility.
+        while (dynamicParam.getIndex() >= dynamicParamSqlNodes.size()) {
+            dynamicParamSqlNodes.add(null);
+        }
+
+        dynamicParamSqlNodes.set(dynamicParam.getIndex(), dynamicParam);
+        return rexBuilder.makeDynamicParam(
+                getDynamicParamType(dynamicParam.getIndex()), dynamicParam.getIndex());
+    }
+
+    /**
+     * Creates a list of collations required to implement the ORDER BY clause, if there is one.
+     * Populates <code>extraOrderExprs</code> with any sort expressions which are not in the select
+     * clause.
+     *
+     * @param bb Scope within which to resolve identifiers
+     * @param select Select clause. Never null, because we invent a dummy SELECT if ORDER BY is
+     *     applied to a set operation (UNION etc.)
+     * @param orderList Order by clause, may be null
+     * @param extraOrderExprs Sort expressions which are not in the select clause (output)
+     * @param collationList List of collations (output)
+     */
+    protected void gatherOrderExprs(
+            Blackboard bb,
+            SqlSelect select,
+            SqlNodeList orderList,
+            List<SqlNode> extraOrderExprs,
+            List<RelFieldCollation> collationList) {
+        // TODO:  add validation rules to SqlValidator also
+        assert bb.root != null : "precondition: child != null";
+        assert select != null;
+        if (orderList == null) {
+            return;
+        }
+
+        if (removeSortInSubQuery(bb.top)) {
+            SqlNode offset = select.getOffset();
+            if ((offset == null
+                            || (offset instanceof SqlLiteral
+                                    && ((SqlLiteral) offset)
+                                            .bigDecimalValue()
+                                            .equals(BigDecimal.ZERO)))
+                    && select.getFetch() == null) {
+                return;
+            }
+        }
+
+        for (SqlNode orderItem : orderList) {
+            collationList.add(
+                    convertOrderItem(
+                            select,
+                            orderItem,
+                            extraOrderExprs,
+                            RelFieldCollation.Direction.ASCENDING,
+                            RelFieldCollation.NullDirection.UNSPECIFIED));
+        }
+    }
+
+    protected RelFieldCollation convertOrderItem(
+            SqlSelect select,
+            SqlNode orderItem,
+            List<SqlNode> extraExprs,
+            RelFieldCollation.Direction direction,
+            RelFieldCollation.NullDirection nullDirection) {
+        assert select != null;
+        // Handle DESC keyword, e.g. 'select a, b from t order by a desc'.
+        switch (orderItem.getKind()) {
+            case DESCENDING:
+                return convertOrderItem(
+                        select,
+                        ((SqlCall) orderItem).operand(0),
+                        extraExprs,
+                        RelFieldCollation.Direction.DESCENDING,
+                        nullDirection);
+            case NULLS_FIRST:
+                return convertOrderItem(
+                        select,
+                        ((SqlCall) orderItem).operand(0),
+                        extraExprs,
+                        direction,
+                        RelFieldCollation.NullDirection.FIRST);
+            case NULLS_LAST:
+                return convertOrderItem(
+                        select,
+                        ((SqlCall) orderItem).operand(0),
+                        extraExprs,
+                        direction,
+                        RelFieldCollation.NullDirection.LAST);
+        }
+
+        SqlNode converted = validator.expandOrderExpr(select, orderItem);
+
+        switch (nullDirection) {
+            case UNSPECIFIED:
+                nullDirection =
+                        validator.config().defaultNullCollation().last(desc(direction))
+                                ? RelFieldCollation.NullDirection.LAST
+                                : RelFieldCollation.NullDirection.FIRST;
+        }
+
+        // Scan the select list and order exprs for an identical expression.
+        final SelectScope selectScope = validator.getRawSelectScope(select);
+        int ordinal = -1;
+        for (SqlNode selectItem : selectScope.getExpandedSelectList()) {
+            ++ordinal;
+            if (converted.equalsDeep(stripAs(selectItem), Litmus.IGNORE)) {
+                return new RelFieldCollation(ordinal, direction, nullDirection);
+            }
+        }
+
+        for (SqlNode extraExpr : extraExprs) {
+            ++ordinal;
+            if (converted.equalsDeep(extraExpr, Litmus.IGNORE)) {
+                return new RelFieldCollation(ordinal, direction, nullDirection);
+            }
+        }
+
+        // TODO:  handle collation sequence
+        // TODO: flag expressions as non-standard
+
+        extraExprs.add(converted);
+        return new RelFieldCollation(ordinal + 1, direction, nullDirection);
+    }
+
+    private static boolean desc(RelFieldCollation.Direction direction) {
+        switch (direction) {
+            case DESCENDING:
+            case STRICTLY_DESCENDING:
+                return true;
+            default:
+                return false;
+        }
+    }
+
+    @Deprecated // to be removed before 2.0
+    protected boolean enableDecorrelation() {
+        // disable sub-query decorrelation when needed.
+        // e.g. if outer joins are not supported.
+        return config.isDecorrelationEnabled();
+    }
+
+    protected RelNode decorrelateQuery(RelNode rootRel) {
+        return RelDecorrelator.decorrelateQuery(rootRel, relBuilder);
+    }
+
+    /**
+     * Returns whether to trim unused fields as part of the conversion process.
+     *
+     * @return Whether to trim unused fields
+     */
+    @Deprecated // to be removed before 2.0
+    public boolean isTrimUnusedFields() {
+        return config.isTrimUnusedFields();
+    }
+
+    /**
+     * Recursively converts a query to a relational expression.
+     *
+     * @param query Query
+     * @param top Whether this query is the top-level query of the statement
+     * @param targetRowType Target row type, or null
+     * @return Relational expression
+     */
+    protected RelRoot convertQueryRecursive(SqlNode query, boolean top, RelDataType targetRowType) {
+        final SqlKind kind = query.getKind();
+        switch (kind) {
+            case SELECT:
+                return RelRoot.of(convertSelect((SqlSelect) query, top), kind);
+            case INSERT:
+                return RelRoot.of(convertInsert((SqlInsert) query), kind);
+            case DELETE:
+                return RelRoot.of(convertDelete((SqlDelete) query), kind);
+            case UPDATE:
+                return RelRoot.of(convertUpdate((SqlUpdate) query), kind);
+            case MERGE:
+                return RelRoot.of(convertMerge((SqlMerge) query), kind);
+            case UNION:
+            case INTERSECT:
+            case EXCEPT:
+                return RelRoot.of(convertSetOp((SqlCall) query), kind);
+            case WITH:
+                return convertWith((SqlWith) query, top);
+            case VALUES:
+                return RelRoot.of(convertValues((SqlCall) query, targetRowType), kind);
+            default:
+                throw new AssertionError("not a query: " + query);
+        }
+    }
+
+    /**
+     * Converts a set operation (UNION, INTERSECT, MINUS) into relational expressions.
+     *
+     * @param call Call to set operator
+     * @return Relational expression
+     */
+    protected RelNode convertSetOp(SqlCall call) {
+        final RelNode left = convertQueryRecursive(call.operand(0), false, null).project();
+        final RelNode right = convertQueryRecursive(call.operand(1), false, null).project();
+        switch (call.getKind()) {
+            case UNION:
+                return LogicalUnion.create(
+                        com.google.common.collect.ImmutableList.of(left, right), all(call));
+
+            case INTERSECT:
+                return LogicalIntersect.create(
+                        com.google.common.collect.ImmutableList.of(left, right), all(call));
+
+            case EXCEPT:
+                return LogicalMinus.create(
+                        com.google.common.collect.ImmutableList.of(left, right), all(call));
+
+            default:
+                throw Util.unexpected(call.getKind());
+        }
+    }
+
+    private boolean all(SqlCall call) {
+        return ((SqlSetOperator) call.getOperator()).isAll();
+    }
+
+    protected RelNode convertInsert(SqlInsert call) {
+        RelOptTable targetTable = getTargetTable(call);
+
+        final RelDataType targetRowType = validator.getValidatedNodeType(call);
+        assert targetRowType != null;
+        RelNode sourceRel = convertQueryRecursive(call.getSource(), true, targetRowType).project();
+        RelNode massagedRel = convertColumnList(call, sourceRel);
+
+        return createModify(targetTable, massagedRel);
+    }
+
+    /** Creates a relational expression to modify a table or modifiable view. */
+    private RelNode createModify(RelOptTable targetTable, RelNode source) {
+        final ModifiableTable modifiableTable = targetTable.unwrap(ModifiableTable.class);
+        if (modifiableTable != null && modifiableTable == targetTable.unwrap(Table.class)) {
+            return modifiableTable.toModificationRel(
+                    cluster,
+                    targetTable,
+                    catalogReader,
+                    source,
+                    LogicalTableModify.Operation.INSERT,
+                    null,
+                    null,
+                    false);
+        }
+        final ModifiableView modifiableView = targetTable.unwrap(ModifiableView.class);
+        if (modifiableView != null) {
+            final Table delegateTable = modifiableView.getTable();
+            final RelDataType delegateRowType = delegateTable.getRowType(typeFactory);
+            final RelOptTable delegateRelOptTable =
+                    RelOptTableImpl.create(
+                            null, delegateRowType, delegateTable, modifiableView.getTablePath());
+            final RelNode newSource =
+                    createSource(targetTable, source, modifiableView, delegateRowType);
+            return createModify(delegateRelOptTable, newSource);
+        }
+        return LogicalTableModify.create(
+                targetTable,
+                catalogReader,
+                source,
+                LogicalTableModify.Operation.INSERT,
+                null,
+                null,
+                false);
+    }
+
+    /**
+     * Wraps a relational expression in the projects and filters implied by a {@link
+     * ModifiableView}.
+     *
+     * <p>The input relational expression is suitable for inserting into the view, and the returned
+     * relational expression is suitable for inserting into its delegate table.
+     *
+     * <p>In principle, the delegate table of a view might be another modifiable view, and if so,
+     * the process can be repeated.
+     */
+    private RelNode createSource(
+            RelOptTable targetTable,
+            RelNode source,
+            ModifiableView modifiableView,
+            RelDataType delegateRowType) {
+        final ImmutableIntList mapping = modifiableView.getColumnMapping();
+        assert mapping.size() == targetTable.getRowType().getFieldCount();
+
+        // For columns represented in the mapping, the expression is just a field
+        // reference.
+        final Map<Integer, RexNode> projectMap = new HashMap<>();
+        final List<RexNode> filters = new ArrayList<>();
+        for (int i = 0; i < mapping.size(); i++) {
+            int target = mapping.get(i);
+            if (target >= 0) {
+                projectMap.put(target, RexInputRef.of(i, source.getRowType()));
+            }
+        }
+
+        // For columns that are not in the mapping, and have a constraint of the
+        // form "column = value", the expression is the literal "value".
+        //
+        // If a column has multiple constraints, the extra ones will become a
+        // filter.
+        final RexNode constraint = modifiableView.getConstraint(rexBuilder, delegateRowType);
+        RelOptUtil.inferViewPredicates(projectMap, filters, constraint);
+        final List<Pair<RexNode, String>> projects = new ArrayList<>();
+        for (RelDataTypeField field : delegateRowType.getFieldList()) {
+            RexNode node = projectMap.get(field.getIndex());
+            if (node == null) {
+                node = rexBuilder.makeNullLiteral(field.getType());
+            }
+            projects.add(
+                    Pair.of(rexBuilder.ensureType(field.getType(), node, false), field.getName()));
+        }
+
+        return relBuilder
+                .push(source)
+                .projectNamed(Pair.left(projects), Pair.right(projects), false)
+                .filter(filters)
+                .build();
+    }
+
+    private RelOptTable.ToRelContext createToRelContext(List<RelHint> hints) {
+        return ViewExpanders.toRelContext(viewExpander, cluster, hints);
+    }
+
+    public RelNode toRel(final RelOptTable table, @Nonnull final List<RelHint> hints) {
+        final RelNode scan = table.toRel(createToRelContext(hints));
+
+        final InitializerExpressionFactory ief =
+                Util.first(
+                        table.unwrap(InitializerExpressionFactory.class),
+                        NullInitializerExpressionFactory.INSTANCE);
+
+        boolean hasVirtualFields =
+                table.getRowType().getFieldList().stream()
+                        .anyMatch(
+                                f ->
+                                        ief.generationStrategy(table, f.getIndex())
+                                                == ColumnStrategy.VIRTUAL);
+
+        if (hasVirtualFields) {
+            final RexNode sourceRef = rexBuilder.makeRangeReference(scan);
+            final Blackboard bb =
+                    createInsertBlackboard(table, sourceRef, table.getRowType().getFieldNames());
+            final List<RexNode> list = new ArrayList<>();
+            for (RelDataTypeField f : table.getRowType().getFieldList()) {
+                final ColumnStrategy strategy = ief.generationStrategy(table, f.getIndex());
+                switch (strategy) {
+                    case VIRTUAL:
+                        list.add(ief.newColumnDefaultValue(table, f.getIndex(), bb));
+                        break;
+                    default:
+                        list.add(
+                                rexBuilder.makeInputRef(
+                                        scan, RelOptTableImpl.realOrdinal(table, f.getIndex())));
+                }
+            }
+            relBuilder.push(scan);
+            relBuilder.project(list);
+            final RelNode project = relBuilder.build();
+            if (ief.postExpressionConversionHook() != null) {
+                return ief.postExpressionConversionHook().apply(bb, project);
+            } else {
+                return project;
+            }
+        }
+
+        return scan;
+    }
+
+    protected RelOptTable getTargetTable(SqlNode call) {
+        final SqlValidatorNamespace targetNs = validator.getNamespace(call);
+        if (targetNs.isWrapperFor(SqlValidatorImpl.DmlNamespace.class)) {
+            final SqlValidatorImpl.DmlNamespace dmlNamespace =
+                    targetNs.unwrap(SqlValidatorImpl.DmlNamespace.class);
+            return SqlValidatorUtil.getRelOptTable(dmlNamespace, catalogReader, null, null);
+        }
+        final SqlValidatorNamespace resolvedNamespace = targetNs.resolve();
+        return SqlValidatorUtil.getRelOptTable(resolvedNamespace, catalogReader, null, null);
+    }
+
+    /**
+     * Creates a source for an INSERT statement.
+     *
+     * <p>If the column list is not specified, source expressions match target columns in order.
+     *
+     * <p>If the column list is specified, Source expressions are mapped to target columns by name
+     * via targetColumnList, and may not cover the entire target table. So, we'll make up a full
+     * row, using a combination of default values and the source expressions provided.
+     *
+     * @param call Insert expression
+     * @param source Source relational expression
+     * @return Converted INSERT statement
+     */
+    protected RelNode convertColumnList(final SqlInsert call, RelNode source) {
+        RelDataType sourceRowType = source.getRowType();
+        final RexNode sourceRef = rexBuilder.makeRangeReference(sourceRowType, 0, false);
+        final List<String> targetColumnNames = new ArrayList<>();
+        final List<RexNode> columnExprs = new ArrayList<>();
+        collectInsertTargets(call, sourceRef, targetColumnNames, columnExprs);
+
+        final RelOptTable targetTable = getTargetTable(call);
+        final RelDataType targetRowType = RelOptTableImpl.realRowType(targetTable);
+        final List<RelDataTypeField> targetFields = targetRowType.getFieldList();
+        final List<RexNode> sourceExps =
+                new ArrayList<>(Collections.nCopies(targetFields.size(), null));
+        final List<String> fieldNames =
+                new ArrayList<>(Collections.nCopies(targetFields.size(), null));
+
+        final InitializerExpressionFactory initializerFactory =
+                getInitializerFactory(validator.getNamespace(call).getTable());
+
+        // Walk the name list and place the associated value in the
+        // expression list according to the ordinal value returned from
+        // the table construct, leaving nulls in the list for columns
+        // that are not referenced.
+        final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
+        for (Pair<String, RexNode> p : Pair.zip(targetColumnNames, columnExprs)) {
+            RelDataTypeField field = nameMatcher.field(targetRowType, p.left);
+            assert field != null : "column " + p.left + " not found";
+            sourceExps.set(field.getIndex(), p.right);
+        }
+
+        // Lazily create a blackboard that contains all non-generated columns.
+        final Supplier<Blackboard> bb =
+                () -> createInsertBlackboard(targetTable, sourceRef, targetColumnNames);
+
+        // Walk the expression list and get default values for any columns
+        // that were not supplied in the statement. Get field names too.
+        for (int i = 0; i < targetFields.size(); ++i) {
+            final RelDataTypeField field = targetFields.get(i);
+            final String fieldName = field.getName();
+            fieldNames.set(i, fieldName);
+            if (sourceExps.get(i) == null || sourceExps.get(i).getKind() == SqlKind.DEFAULT) {
+                sourceExps.set(
+                        i, initializerFactory.newColumnDefaultValue(targetTable, i, bb.get()));
+
+                // bare nulls are dangerous in the wrong hands
+                sourceExps.set(i, castNullLiteralIfNeeded(sourceExps.get(i), field.getType()));
+            }
+        }
+
+        return relBuilder.push(source).projectNamed(sourceExps, fieldNames, false).build();
+    }
+
+    /**
+     * Creates a blackboard for translating the expressions of generated columns in an INSERT
+     * statement.
+     */
+    private Blackboard createInsertBlackboard(
+            RelOptTable targetTable, RexNode sourceRef, List<String> targetColumnNames) {
+        final Map<String, RexNode> nameToNodeMap = new HashMap<>();
+        int j = 0;
+
+        // Assign expressions for non-generated columns.
+        final List<ColumnStrategy> strategies = targetTable.getColumnStrategies();
+        final List<String> targetFields = targetTable.getRowType().getFieldNames();
+        for (String targetColumnName : targetColumnNames) {
+            final int i = targetFields.indexOf(targetColumnName);
+            switch (strategies.get(i)) {
+                case STORED:
+                case VIRTUAL:
+                    break;
+                default:
+                    nameToNodeMap.put(targetColumnName, rexBuilder.makeFieldAccess(sourceRef, j++));
+            }
+        }
+        return createBlackboard(null, nameToNodeMap, false);
+    }
+
+    private InitializerExpressionFactory getInitializerFactory(SqlValidatorTable validatorTable) {
+        // We might unwrap a null instead of a InitializerExpressionFactory.
+        final Table table = unwrap(validatorTable, Table.class);
+        if (table != null) {
+            InitializerExpressionFactory f = unwrap(table, InitializerExpressionFactory.class);
+            if (f != null) {
+                return f;
+            }
+        }
+        return NullInitializerExpressionFactory.INSTANCE;
+    }
+
+    private static <T> T unwrap(Object o, Class<T> clazz) {
+        if (o instanceof Wrapper) {
+            return ((Wrapper) o).unwrap(clazz);
+        }
+        return null;
+    }
+
+    private RexNode castNullLiteralIfNeeded(RexNode node, RelDataType type) {
+        if (!RexLiteral.isNullLiteral(node)) {
+            return node;
+        }
+        return rexBuilder.makeCast(type, node);
+    }
+
+    /**
+     * Given an INSERT statement, collects the list of names to be populated and the expressions to
+     * put in them.
+     *
+     * @param call Insert statement
+     * @param sourceRef Expression representing a row from the source relational expression
+     * @param targetColumnNames List of target column names, to be populated
+     * @param columnExprs List of expressions, to be populated
+     */
+    protected void collectInsertTargets(
+            SqlInsert call,
+            final RexNode sourceRef,
+            final List<String> targetColumnNames,
+            List<RexNode> columnExprs) {
+        final RelOptTable targetTable = getTargetTable(call);
+        final RelDataType tableRowType = targetTable.getRowType();
+        SqlNodeList targetColumnList = call.getTargetColumnList();
+        if (targetColumnList == null) {
+            if (validator.config().sqlConformance().isInsertSubsetColumnsAllowed()) {
+                final RelDataType targetRowType =
+                        typeFactory.createStructType(
+                                tableRowType
+                                        .getFieldList()
+                                        .subList(0, sourceRef.getType().getFieldCount()));
+                targetColumnNames.addAll(targetRowType.getFieldNames());
+            } else {
+                targetColumnNames.addAll(tableRowType.getFieldNames());
+            }
+        } else {
+            for (int i = 0; i < targetColumnList.size(); i++) {
+                SqlIdentifier id = (SqlIdentifier) targetColumnList.get(i);
+                RelDataTypeField field =
+                        SqlValidatorUtil.getTargetField(
+                                tableRowType, typeFactory, id, catalogReader, targetTable);
+                assert field != null : "column " + id.toString() + " not found";
+                targetColumnNames.add(field.getName());
+            }
+        }
+
+        final Blackboard bb = createInsertBlackboard(targetTable, sourceRef, targetColumnNames);
+
+        // Next, assign expressions for generated columns.
+        final List<ColumnStrategy> strategies = targetTable.getColumnStrategies();
+        for (String columnName : targetColumnNames) {
+            final int i = tableRowType.getFieldNames().indexOf(columnName);
+            final RexNode expr;
+            switch (strategies.get(i)) {
+                case STORED:
+                    final InitializerExpressionFactory f =
+                            Util.first(
+                                    targetTable.unwrap(InitializerExpressionFactory.class),
+                                    NullInitializerExpressionFactory.INSTANCE);
+                    expr = f.newColumnDefaultValue(targetTable, i, bb);
+                    break;
+                case VIRTUAL:
+                    expr = null;
+                    break;
+                default:
+                    expr = bb.nameToNodeMap.get(columnName);
+            }
+            columnExprs.add(expr);
+        }
+
+        // Remove virtual columns from the list.
+        for (int i = 0; i < targetColumnNames.size(); i++) {
+            if (columnExprs.get(i) == null) {
+                columnExprs.remove(i);
+                targetColumnNames.remove(i);
+                --i;
+            }
+        }
+    }
+
+    private RelNode convertDelete(SqlDelete call) {
+        RelOptTable targetTable = getTargetTable(call);
+        RelNode sourceRel = convertSelect(call.getSourceSelect(), false);
+        return LogicalTableModify.create(
+                targetTable,
+                catalogReader,
+                sourceRel,
+                LogicalTableModify.Operation.DELETE,
+                null,
+                null,
+                false);
+    }
+
+    private RelNode convertUpdate(SqlUpdate call) {
+        final SqlValidatorScope scope = validator.getWhereScope(call.getSourceSelect());
+        Blackboard bb = createBlackboard(scope, null, false);
+
+        replaceSubQueries(bb, call, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+        RelOptTable targetTable = getTargetTable(call);
+
+        // convert update column list from SqlIdentifier to String
+        final List<String> targetColumnNameList = new ArrayList<>();
+        final RelDataType targetRowType = targetTable.getRowType();
+        for (SqlNode node : call.getTargetColumnList()) {
+            SqlIdentifier id = (SqlIdentifier) node;
+            RelDataTypeField field =
+                    SqlValidatorUtil.getTargetField(
+                            targetRowType, typeFactory, id, catalogReader, targetTable);
+            assert field != null : "column " + id.toString() + " not found";
+            targetColumnNameList.add(field.getName());
+        }
+
+        RelNode sourceRel = convertSelect(call.getSourceSelect(), false);
+
+        bb.setRoot(sourceRel, false);
+        com.google.common.collect.ImmutableList.Builder<RexNode>
+                rexNodeSourceExpressionListBuilder =
+                        com.google.common.collect.ImmutableList.builder();
+        for (SqlNode n : call.getSourceExpressionList()) {
+            RexNode rn = bb.convertExpression(n);
+            rexNodeSourceExpressionListBuilder.add(rn);
+        }
+
+        return LogicalTableModify.create(
+                targetTable,
+                catalogReader,
+                sourceRel,
+                LogicalTableModify.Operation.UPDATE,
+                targetColumnNameList,
+                rexNodeSourceExpressionListBuilder.build(),
+                false);
+    }
+
+    private RelNode convertMerge(SqlMerge call) {
+        RelOptTable targetTable = getTargetTable(call);
+
+        // convert update column list from SqlIdentifier to String
+        final List<String> targetColumnNameList = new ArrayList<>();
+        final RelDataType targetRowType = targetTable.getRowType();
+        SqlUpdate updateCall = call.getUpdateCall();
+        if (updateCall != null) {
+            for (SqlNode targetColumn : updateCall.getTargetColumnList()) {
+                SqlIdentifier id = (SqlIdentifier) targetColumn;
+                RelDataTypeField field =
+                        SqlValidatorUtil.getTargetField(
+                                targetRowType, typeFactory, id, catalogReader, targetTable);
+                assert field != null : "column " + id.toString() + " not found";
+                targetColumnNameList.add(field.getName());
+            }
+        }
+
+        // replace the projection of the source select with a
+        // projection that contains the following:
+        // 1) the expressions corresponding to the new insert row (if there is
+        //    an insert)
+        // 2) all columns from the target table (if there is an update)
+        // 3) the set expressions in the update call (if there is an update)
+
+        // first, convert the merge's source select to construct the columns
+        // from the target table and the set expressions in the update call
+        RelNode mergeSourceRel = convertSelect(call.getSourceSelect(), false);
+
+        // then, convert the insert statement so we can get the insert
+        // values expressions
+        SqlInsert insertCall = call.getInsertCall();
+        int nLevel1Exprs = 0;
+        List<RexNode> level1InsertExprs = null;
+        List<RexNode> level2InsertExprs = null;
+        if (insertCall != null) {
+            RelNode insertRel = convertInsert(insertCall);
+
+            // if there are 2 level of projections in the insert source, combine
+            // them into a single project; level1 refers to the topmost project;
+            // the level1 projection contains references to the level2
+            // expressions, except in the case where no target expression was
+            // provided, in which case, the expression is the default value for
+            // the column; or if the expressions directly map to the source
+            // table
+            level1InsertExprs = ((LogicalProject) insertRel.getInput(0)).getProjects();
+            if (insertRel.getInput(0).getInput(0) instanceof LogicalProject) {
+                level2InsertExprs =
+                        ((LogicalProject) insertRel.getInput(0).getInput(0)).getProjects();
+            }
+            nLevel1Exprs = level1InsertExprs.size();
+        }
+
+        LogicalJoin join = (LogicalJoin) mergeSourceRel.getInput(0);
+        int nSourceFields = join.getLeft().getRowType().getFieldCount();
+        final List<RexNode> projects = new ArrayList<>();
+        for (int level1Idx = 0; level1Idx < nLevel1Exprs; level1Idx++) {
+            if ((level2InsertExprs != null)
+                    && (level1InsertExprs.get(level1Idx) instanceof RexInputRef)) {
+                int level2Idx = ((RexInputRef) level1InsertExprs.get(level1Idx)).getIndex();
+                projects.add(level2InsertExprs.get(level2Idx));
+            } else {
+                projects.add(level1InsertExprs.get(level1Idx));
+            }
+        }
+        if (updateCall != null) {
+            final LogicalProject project = (LogicalProject) mergeSourceRel;
+            projects.addAll(Util.skip(project.getProjects(), nSourceFields));
+        }
+
+        relBuilder.push(join).project(projects);
+
+        return LogicalTableModify.create(
+                targetTable,
+                catalogReader,
+                relBuilder.build(),
+                LogicalTableModify.Operation.MERGE,
+                targetColumnNameList,
+                null,
+                false);
+    }
+
+    /**
+     * Converts an identifier into an expression in a given scope. For example, the "empno" in
+     * "select empno from emp join dept" becomes "emp.empno".
+     */
+    private RexNode convertIdentifier(Blackboard bb, SqlIdentifier identifier) {
+        // first check for reserved identifiers like CURRENT_USER
+        final SqlCall call = bb.getValidator().makeNullaryCall(identifier);
+        if (call != null) {
+            return bb.convertExpression(call);
+        }
+
+        String pv = null;
+        if (bb.isPatternVarRef && identifier.names.size() > 1) {
+            pv = identifier.names.get(0);
+        }
+
+        final SqlQualified qualified;
+        if (bb.scope != null) {
+            qualified = bb.scope.fullyQualify(identifier);
+        } else {
+            qualified = SqlQualified.create(null, 1, null, identifier);
+        }
+        final Pair<RexNode, Map<String, Integer>> e0 = bb.lookupExp(qualified);
+        RexNode e = e0.left;
+        for (String name : qualified.suffix()) {
+            if (e == e0.left && e0.right != null) {
+                int i = e0.right.get(name);
+                e = rexBuilder.makeFieldAccess(e, i);
+            } else {
+                final boolean caseSensitive = true; // name already fully-qualified
+                if (identifier.isStar() && bb.scope instanceof MatchRecognizeScope) {
+                    e = rexBuilder.makeFieldAccess(e, 0);
+                } else {
+                    e = rexBuilder.makeFieldAccess(e, name, caseSensitive);
+                }
+            }
+        }
+        if (e instanceof RexInputRef) {
+            // adjust the type to account for nulls introduced by outer joins
+            e = adjustInputRef(bb, (RexInputRef) e);
+            if (pv != null) {
+                e = RexPatternFieldRef.of(pv, (RexInputRef) e);
+            }
+        }
+
+        if (e0.left instanceof RexCorrelVariable) {
+            assert e instanceof RexFieldAccess;
+            final RexNode prev =
+                    bb.mapCorrelateToRex.put(((RexCorrelVariable) e0.left).id, (RexFieldAccess) e);
+            assert prev == null;
+        }
+        return e;
+    }
+
+    /**
+     * Adjusts the type of a reference to an input field to account for nulls introduced by outer
+     * joins; and adjusts the offset to match the physical implementation.
+     *
+     * @param bb Blackboard
+     * @param inputRef Input ref
+     * @return Adjusted input ref
+     */
+    protected RexNode adjustInputRef(Blackboard bb, RexInputRef inputRef) {
+        RelDataTypeField field = bb.getRootField(inputRef);
+        if (field != null) {
+            if (!SqlTypeUtil.equalSansNullability(
+                    typeFactory, field.getType(), inputRef.getType())) {
+                return inputRef;
+            }
+            return rexBuilder.makeInputRef(field.getType(), inputRef.getIndex());
+        }
+        return inputRef;
+    }
+
+    /**
+     * Converts a row constructor into a relational expression.
+     *
+     * @param bb Blackboard
+     * @param rowConstructor Row constructor expression
+     * @return Relational expression which returns a single row.
+     */
+    private RelNode convertRowConstructor(Blackboard bb, SqlCall rowConstructor) {
+        com.google.common.base.Preconditions.checkArgument(isRowConstructor(rowConstructor));
+        final List<SqlNode> operands = rowConstructor.getOperandList();
+        return convertMultisets(operands, bb);
+    }
+
+    private RelNode convertCursor(Blackboard bb, SubQuery subQuery) {
+        final SqlCall cursorCall = (SqlCall) subQuery.node;
+        assert cursorCall.operandCount() == 1;
+        SqlNode query = cursorCall.operand(0);
+        RelNode converted = convertQuery(query, false, false).rel;
+        int iCursor = bb.cursors.size();
+        bb.cursors.add(converted);
+        subQuery.expr = new RexInputRef(iCursor, converted.getRowType());
+        return converted;
+    }
+
+    private RelNode convertMultisets(final List<SqlNode> operands, Blackboard bb) {
+        // NOTE: Wael 2/04/05: this implementation is not the most efficient in
+        // terms of planning since it generates XOs that can be reduced.
+        final List<Object> joinList = new ArrayList<>();
+        List<SqlNode> lastList = new ArrayList<>();
+        for (int i = 0; i < operands.size(); i++) {
+            SqlNode operand = operands.get(i);
+            if (!(operand instanceof SqlCall)) {
+                lastList.add(operand);
+                continue;
+            }
+
+            final SqlCall call = (SqlCall) operand;
+            final RelNode input;
+            switch (call.getKind()) {
+                case MULTISET_VALUE_CONSTRUCTOR:
+                case ARRAY_VALUE_CONSTRUCTOR:
+                    final SqlNodeList list =
+                            new SqlNodeList(call.getOperandList(), call.getParserPosition());
+                    CollectNamespace nss = (CollectNamespace) validator.getNamespace(call);
+                    Blackboard usedBb;
+                    if (null != nss) {
+                        usedBb = createBlackboard(nss.getScope(), null, false);
+                    } else {
+                        usedBb =
+                                createBlackboard(
+                                        new ListScope(bb.scope) {
+                                            public SqlNode getNode() {
+                                                return call;
+                                            }
+                                        },
+                                        null,
+                                        false);
+                    }
+                    RelDataType multisetType = validator.getValidatedNodeType(call);
+                    validator.setValidatedNodeType(list, multisetType.getComponentType());
+                    input = convertQueryOrInList(usedBb, list, null);
+                    break;
+                case MULTISET_QUERY_CONSTRUCTOR:
+                case ARRAY_QUERY_CONSTRUCTOR:
+                    final RelRoot root = convertQuery(call.operand(0), false, true);
+                    input = root.rel;
+                    break;
+                default:
+                    lastList.add(operand);
+                    continue;
+            }
+
+            if (lastList.size() > 0) {
+                joinList.add(lastList);
+            }
+            lastList = new ArrayList<>();
+            Collect collect =
+                    new Collect(
+                            cluster,
+                            cluster.traitSetOf(Convention.NONE),
+                            input,
+                            validator.deriveAlias(call, i));
+            joinList.add(collect);
+        }
+
+        if (joinList.size() == 0) {
+            joinList.add(lastList);
+        }
+
+        for (int i = 0; i < joinList.size(); i++) {
+            Object o = joinList.get(i);
+            if (o instanceof List) {
+                @SuppressWarnings("unchecked")
+                List<SqlNode> projectList = (List<SqlNode>) o;
+                final List<RexNode> selectList = new ArrayList<>();
+                final List<String> fieldNameList = new ArrayList<>();
+                for (int j = 0; j < projectList.size(); j++) {
+                    SqlNode operand = projectList.get(j);
+                    selectList.add(bb.convertExpression(operand));
+
+                    // REVIEW angel 5-June-2005: Use deriveAliasFromOrdinal
+                    // instead of deriveAlias to match field names from
+                    // SqlRowOperator. Otherwise, get error   Type
+                    // 'RecordType(INTEGER EMPNO)' has no field 'EXPR$0' when
+                    // doing   select * from unnest(     select multiset[empno]
+                    // from sales.emps);
+
+                    fieldNameList.add(SqlUtil.deriveAliasFromOrdinal(j));
+                }
+
+                relBuilder
+                        .push(LogicalValues.createOneRow(cluster))
+                        .projectNamed(selectList, fieldNameList, true);
+
+                joinList.set(i, relBuilder.build());
+            }
+        }
+
+        RelNode ret = (RelNode) joinList.get(0);
+        for (int i = 1; i < joinList.size(); i++) {
+            RelNode relNode = (RelNode) joinList.get(i);
+            ret =
+                    RelFactories.DEFAULT_JOIN_FACTORY.createJoin(
+                            ret,
+                            relNode,
+                            com.google.common.collect.ImmutableList.of(),
+                            rexBuilder.makeLiteral(true),
+                            com.google.common.collect.ImmutableSet.of(),
+                            JoinRelType.INNER,
+                            false);
+        }
+        return ret;
+    }
+
+    private void convertSelectList(Blackboard bb, SqlSelect select, List<SqlNode> orderList) {
+        SqlNodeList selectList = select.getSelectList();
+        selectList = validator.expandStar(selectList, select, false);
+
+        replaceSubQueries(bb, selectList, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+
+        List<String> fieldNames = new ArrayList<>();
+        final List<RexNode> exprs = new ArrayList<>();
+        final Collection<String> aliases = new TreeSet<>();
+
+        // Project any system fields. (Must be done before regular select items,
+        // because offsets may be affected.)
+        final List<SqlMonotonicity> columnMonotonicityList = new ArrayList<>();
+        extraSelectItems(bb, select, exprs, fieldNames, aliases, columnMonotonicityList);
+
+        // Project select clause.
+        int i = -1;
+        for (SqlNode expr : selectList) {
+            ++i;
+            exprs.add(bb.convertExpression(expr));
+            fieldNames.add(deriveAlias(expr, aliases, i));
+        }
+
+        // Project extra fields for sorting.
+        for (SqlNode expr : orderList) {
+            ++i;
+            SqlNode expr2 = validator.expandOrderExpr(select, expr);
+            exprs.add(bb.convertExpression(expr2));
+            fieldNames.add(deriveAlias(expr, aliases, i));
+        }
+
+        fieldNames =
+                SqlValidatorUtil.uniquify(
+                        fieldNames, catalogReader.nameMatcher().isCaseSensitive());
+
+        relBuilder.push(bb.root).projectNamed(exprs, fieldNames, true);
+        bb.setRoot(relBuilder.build(), false);
+
+        assert bb.columnMonotonicities.isEmpty();
+        bb.columnMonotonicities.addAll(columnMonotonicityList);
+        for (SqlNode selectItem : selectList) {
+            bb.columnMonotonicities.add(selectItem.getMonotonicity(bb.scope));
+        }
+    }
+
+    /**
+     * Adds extra select items. The default implementation adds nothing; derived classes may add
+     * columns to exprList, nameList, aliasList and columnMonotonicityList.
+     *
+     * @param bb Blackboard
+     * @param select Select statement being translated
+     * @param exprList List of expressions in select clause
+     * @param nameList List of names, one per column
+     * @param aliasList Collection of aliases that have been used already
+     * @param columnMonotonicityList List of monotonicity, one per column
+     */
+    protected void extraSelectItems(
+            Blackboard bb,
+            SqlSelect select,
+            List<RexNode> exprList,
+            List<String> nameList,
+            Collection<String> aliasList,
+            List<SqlMonotonicity> columnMonotonicityList) {}
+
+    private String deriveAlias(final SqlNode node, Collection<String> aliases, final int ordinal) {
+        String alias = validator.deriveAlias(node, ordinal);
+        if ((alias == null) || aliases.contains(alias)) {
+            String aliasBase = (alias == null) ? "EXPR$" : alias;
+            for (int j = 0; ; j++) {
+                alias = aliasBase + j;
+                if (!aliases.contains(alias)) {
+                    break;
+                }
+            }
+        }
+        aliases.add(alias);
+        return alias;
+    }
+
+    /** Converts a WITH sub-query into a relational expression. */
+    public RelRoot convertWith(SqlWith with, boolean top) {
+        return convertQuery(with.body, false, top);
+    }
+
+    /** Converts a SELECT statement's parse tree into a relational expression. */
+    public RelNode convertValues(SqlCall values, RelDataType targetRowType) {
+        final SqlValidatorScope scope = validator.getOverScope(values);
+        assert scope != null;
+        final Blackboard bb = createBlackboard(scope, null, false);
+        convertValuesImpl(bb, values, targetRowType);
+        return bb.root;
+    }
+
+    /**
+     * Converts a values clause (as in "INSERT INTO T(x,y) VALUES (1,2)") into a relational
+     * expression.
+     *
+     * @param bb Blackboard
+     * @param values Call to SQL VALUES operator
+     * @param targetRowType Target row type
+     */
+    private void convertValuesImpl(Blackboard bb, SqlCall values, RelDataType targetRowType) {
+        // Attempt direct conversion to LogicalValues; if that fails, deal with
+        // fancy stuff like sub-queries below.
+        RelNode valuesRel =
+                convertRowValues(bb, values, values.getOperandList(), true, targetRowType);
+        if (valuesRel != null) {
+            bb.setRoot(valuesRel, true);
+            return;
+        }
+
+        final List<RelNode> unionRels = new ArrayList<>();
+        for (SqlNode rowConstructor1 : values.getOperandList()) {
+            SqlCall rowConstructor = (SqlCall) rowConstructor1;
+            Blackboard tmpBb = createBlackboard(bb.scope, null, false);
+            replaceSubQueries(tmpBb, rowConstructor, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
+            final List<Pair<RexNode, String>> exps = new ArrayList<>();
+            for (Ord<SqlNode> operand : Ord.zip(rowConstructor.getOperandList())) {
+                exps.add(
+                        Pair.of(
+                                tmpBb.convertExpression(operand.e),
+                                validator.deriveAlias(operand.e, operand.i)));
+            }
+            RelNode in = (null == tmpBb.root) ? LogicalValues.createOneRow(cluster) : tmpBb.root;
+            unionRels.add(relBuilder.push(in).project(Pair.left(exps), Pair.right(exps)).build());
+        }
+
+        if (unionRels.size() == 0) {
+            throw new AssertionError("empty values clause");
+        } else if (unionRels.size() == 1) {
+            bb.setRoot(unionRels.get(0), true);
+        } else {
+            bb.setRoot(LogicalUnion.create(unionRels, true), true);
+        }
+    }
+
+    // ~ Inner Classes ----------------------------------------------------------
+
+    /** A Tuple to remember all calls to Blackboard.register */
+    private static class RegisterArgs {
+        final RelNode rel;
+        final JoinRelType joinType;
+        final List<RexNode> leftKeys;
+
+        RegisterArgs(RelNode rel, JoinRelType joinType, List<RexNode> leftKeys) {
+            this.rel = rel;
+            this.joinType = joinType;
+            this.leftKeys = leftKeys;
+        }
+    }
+
+    /** Workspace for translating an individual SELECT statement (or sub-SELECT). */
+    protected class Blackboard implements SqlRexContext, SqlVisitor<RexNode>, InitializerContext {
+        /** Collection of {@link RelNode} objects which correspond to a SELECT statement. */
+        public final SqlValidatorScope scope;
+
+        private final Map<String, RexNode> nameToNodeMap;
+        public RelNode root;
+        private List<RelNode> inputs;
+        private final Map<CorrelationId, RexFieldAccess> mapCorrelateToRex = new HashMap<>();
+        private List<RegisterArgs> registered = new ArrayList<>();
+
+        private boolean isPatternVarRef = false;
+
+        final List<RelNode> cursors = new ArrayList<>();
+
+        /**
+         * List of <code>IN</code> and <code>EXISTS</code> nodes inside this <code>SELECT</code>
+         * statement (but not inside sub-queries).
+         */
+        private final List<SubQuery> subQueryList = new ArrayList<>();
+
+        /** Workspace for building aggregates. */
+        AggConverter agg;
+
+        /**
+         * When converting window aggregate, we need to know if the window is guaranteed to be
+         * non-empty.
+         */
+        SqlWindow window;
+
+        /**
+         * Project the groupby expressions out of the root of this sub-select. Sub-queries can
+         * reference group by expressions projected from the "right" to the sub-query.
+         */
+        private final Map<RelNode, Map<Integer, Integer>> mapRootRelToFieldProjection =
+                new HashMap<>();
+
+        private final List<SqlMonotonicity> columnMonotonicities = new ArrayList<>();
+
+        private final List<RelDataTypeField> systemFieldList = new ArrayList<>();
+        final boolean top;
+
+        private final InitializerExpressionFactory initializerExpressionFactory =
+                new NullInitializerExpressionFactory();
+
+        /**
+         * Creates a Blackboard.
+         *
+         * @param scope Name-resolution scope for expressions validated within this query. Can be
+         *     null if this Blackboard is for a leaf node, say
+         * @param nameToNodeMap Map which translates the expression to map a given parameter into,
+         *     if translating expressions; null otherwise
+         * @param top Whether this is the root of the query
+         */
+        protected Blackboard(
+                SqlValidatorScope scope, Map<String, RexNode> nameToNodeMap, boolean top) {
+            this.scope = scope;
+            this.nameToNodeMap = nameToNodeMap;
+            this.top = top;
+        }
+
+        public void setPatternVarRef(boolean isVarRef) {
+            this.isPatternVarRef = isVarRef;
+        }
+
+        public RexNode register(RelNode rel, JoinRelType joinType) {
+            return register(rel, joinType, null);
+        }
+
+        /**
+         * Registers a relational expression.
+         *
+         * @param rel Relational expression
+         * @param joinType Join type
+         * @param leftKeys LHS of IN clause, or null for expressions other than IN
+         * @return Expression with which to refer to the row (or partial row) coming from this
+         *     relational expression's side of the join
+         */
+        public RexNode register(RelNode rel, JoinRelType joinType, List<RexNode> leftKeys) {
+            assert joinType != null;
+            registered.add(new RegisterArgs(rel, joinType, leftKeys));
+            if (root == null) {
+                assert leftKeys == null;
+                setRoot(rel, false);
+                return rexBuilder.makeRangeReference(root.getRowType(), 0, false);
+            }
+
+            final RexNode joinCond;
+            final int origLeftInputCount = root.getRowType().getFieldCount();
+            if (leftKeys != null) {
+                List<RexNode> newLeftInputExprs = new ArrayList<>();
+                for (int i = 0; i < origLeftInputCount; i++) {
+                    newLeftInputExprs.add(rexBuilder.makeInputRef(root, i));
+                }
+
+                final List<Integer> leftJoinKeys = new ArrayList<>();
+                for (RexNode leftKey : leftKeys) {
+                    int index = newLeftInputExprs.indexOf(leftKey);
+                    if (index < 0 || joinType == JoinRelType.LEFT) {
+                        index = newLeftInputExprs.size();
+                        newLeftInputExprs.add(leftKey);
+                    }
+                    leftJoinKeys.add(index);
+                }
+
+                RelNode newLeftInput = relBuilder.push(root).project(newLeftInputExprs).build();
+
+                // maintain the group by mapping in the new LogicalProject
+                if (mapRootRelToFieldProjection.containsKey(root)) {
+                    mapRootRelToFieldProjection.put(
+                            newLeftInput, mapRootRelToFieldProjection.get(root));
+                }
+
+                setRoot(newLeftInput, false);
+
+                // right fields appear after the LHS fields.
+                final int rightOffset =
+                        root.getRowType().getFieldCount()
+                                - newLeftInput.getRowType().getFieldCount();
+                final List<Integer> rightKeys =
+                        Util.range(rightOffset, rightOffset + leftKeys.size());
+
+                joinCond =
+                        RelOptUtil.createEquiJoinCondition(
+                                newLeftInput, leftJoinKeys, rel, rightKeys, rexBuilder);
+            } else {
+                joinCond = rexBuilder.makeLiteral(true);
+            }
+
+            int leftFieldCount = root.getRowType().getFieldCount();
+            final RelNode join = createJoin(this, root, rel, joinCond, joinType);
+
+            setRoot(join, false);
+
+            if (leftKeys != null && joinType == JoinRelType.LEFT) {
+                final int leftKeyCount = leftKeys.size();
+                int rightFieldLength = rel.getRowType().getFieldCount();
+                assert leftKeyCount == rightFieldLength - 1;
+
+                final int rexRangeRefLength = leftKeyCount + rightFieldLength;
+                RelDataType returnType =
+                        typeFactory.createStructType(
+                                new AbstractList<Map.Entry<String, RelDataType>>() {
+                                    public Map.Entry<String, RelDataType> get(int index) {
+                                        return join.getRowType()
+                                                .getFieldList()
+                                                .get(origLeftInputCount + index);
+                                    }
+
+                                    public int size() {
+                                        return rexRangeRefLength;
+                                    }
+                                });
+
+                return rexBuilder.makeRangeReference(returnType, origLeftInputCount, false);
+            } else {
+                return rexBuilder.makeRangeReference(
+                        rel.getRowType(), leftFieldCount, joinType.generatesNullsOnRight());
+            }
+        }
+
+        /**
+         * Re-register the {@code registered} with given root node and return the new root node.
+         *
+         * @param root The given root, never leaf
+         * @return new root after the registration
+         */
+        public RelNode reRegister(RelNode root) {
+            setRoot(root, false);
+            List<RegisterArgs> registerCopy = registered;
+            registered = new ArrayList<>();
+            for (RegisterArgs reg : registerCopy) {
+                register(reg.rel, reg.joinType, reg.leftKeys);
+            }
+            return this.root;
+        }
+
+        /**
+         * Sets a new root relational expression, as the translation process backs its way further
+         * up the tree.
+         *
+         * @param root New root relational expression
+         * @param leaf Whether the relational expression is a leaf, that is, derived from an atomic
+         *     relational expression such as a table name in the from clause, or the projection on
+         *     top of a select-sub-query. In particular, relational expressions derived from JOIN
+         *     operators are not leaves, but set expressions are.
+         */
+        public void setRoot(RelNode root, boolean leaf) {
+            setRoot(Collections.singletonList(root), root, root instanceof LogicalJoin);
+            if (leaf) {
+                leaves.put(root, root.getRowType().getFieldCount());
+            }
+            this.columnMonotonicities.clear();
+        }
+
+        private void setRoot(List<RelNode> inputs, RelNode root, boolean hasSystemFields) {
+            this.inputs = inputs;
+            this.root = root;
+            this.systemFieldList.clear();
+            if (hasSystemFields) {
+                this.systemFieldList.addAll(getSystemFields());
+            }
+        }
+
+        /**
+         * Notifies this Blackboard that the root just set using {@link #setRoot(RelNode, boolean)}
+         * was derived using dataset substitution.
+         *
+         * <p>The default implementation is not interested in such notifications, and does nothing.
+         *
+         * @param datasetName Dataset name
+         */
+        public void setDataset(String datasetName) {}
+
+        void setRoot(List<RelNode> inputs) {
+            setRoot(inputs, null, false);
+        }
+
+        /**
+         * Returns an expression with which to reference a from-list item.
+         *
+         * @param qualified the alias of the from item
+         * @return a {@link RexFieldAccess} or {@link RexRangeRef}, or null if not found
+         */
+        Pair<RexNode, Map<String, Integer>> lookupExp(SqlQualified qualified) {
+            if (nameToNodeMap != null && qualified.prefixLength == 1) {
+                RexNode node = nameToNodeMap.get(qualified.identifier.names.get(0));
+                if (node == null) {
+                    throw new AssertionError(
+                            "Unknown identifier '"
+                                    + qualified.identifier
+                                    + "' encountered while expanding expression");
+                }
+                return Pair.of(node, null);
+            }
+            final SqlNameMatcher nameMatcher =
+                    scope.getValidator().getCatalogReader().nameMatcher();
+            final SqlValidatorScope.ResolvedImpl resolved = new SqlValidatorScope.ResolvedImpl();
+            scope.resolve(qualified.prefix(), nameMatcher, false, resolved);
+            if (!(resolved.count() == 1)) {
+                return null;
+            }
+            final SqlValidatorScope.Resolve resolve = resolved.only();
+            final RelDataType rowType = resolve.rowType();
+
+            // Found in current query's from list.  Find which from item.
+            // We assume that the order of the from clause items has been
+            // preserved.
+            final SqlValidatorScope ancestorScope = resolve.scope;
+            boolean isParent = ancestorScope != scope;
+            if ((inputs != null) && !isParent) {
+                final LookupContext rels = new LookupContext(this, inputs, systemFieldList.size());
+                final RexNode node = lookup(resolve.path.steps().get(0).i, rels);
+                if (node == null) {
+                    return null;
+                } else {
+                    final Map<String, Integer> fieldOffsets = new HashMap<>();
+                    for (RelDataTypeField f : resolve.rowType().getFieldList()) {
+                        if (!fieldOffsets.containsKey(f.getName())) {
+                            fieldOffsets.put(f.getName(), f.getIndex());
+                        }
+                    }
+                    final Map<String, Integer> map =
+                            com.google.common.collect.ImmutableMap.copyOf(fieldOffsets);
+                    return Pair.of(node, map);
+                }
+            } else {
+                // We're referencing a relational expression which has not been
+                // converted yet. This occurs when from items are correlated,
+                // e.g. "select from emp as emp join emp.getDepts() as dept".
+                // Create a temporary expression.
+                DeferredLookup lookup = new DeferredLookup(this, qualified.identifier.names.get(0));
+                final CorrelationId correlId = cluster.createCorrel();
+                mapCorrelToDeferred.put(correlId, lookup);
+                if (resolve.path.steps().get(0).i < 0) {
+                    return Pair.of(rexBuilder.makeCorrel(rowType, correlId), null);
+                } else {
+                    final RelDataTypeFactory.Builder builder = typeFactory.builder();
+                    final ListScope ancestorScope1 = (ListScope) resolve.scope;
+                    final com.google.common.collect.ImmutableMap.Builder<String, Integer> fields =
+                            com.google.common.collect.ImmutableMap.builder();
+                    int i = 0;
+                    int offset = 0;
+                    for (SqlValidatorNamespace c : ancestorScope1.getChildren()) {
+                        builder.addAll(c.getRowType().getFieldList());
+                        if (i == resolve.path.steps().get(0).i) {
+                            for (RelDataTypeField field : c.getRowType().getFieldList()) {
+                                fields.put(field.getName(), field.getIndex() + offset);
+                            }
+                        }
+                        ++i;
+                        offset += c.getRowType().getFieldCount();
+                    }
+                    final RexNode c = rexBuilder.makeCorrel(builder.uniquify().build(), correlId);
+                    return Pair.of(c, fields.build());
+                }
+            }
+        }
+
+        /**
+         * Creates an expression with which to reference the expression whose offset in its
+         * from-list is {@code offset}.
+         */
+        RexNode lookup(int offset, LookupContext lookupContext) {
+            Pair<RelNode, Integer> pair = lookupContext.findRel(offset);
+            return rexBuilder.makeRangeReference(pair.left.getRowType(), pair.right, false);
+        }
+
+        RelDataTypeField getRootField(RexInputRef inputRef) {
+            if (inputs == null) {
+                return null;
+            }
+            int fieldOffset = inputRef.getIndex();
+            for (RelNode input : inputs) {
+                RelDataType rowType = input.getRowType();
+                if (fieldOffset < rowType.getFieldCount()) {
+                    return rowType.getFieldList().get(fieldOffset);
+                }
+                fieldOffset -= rowType.getFieldCount();
+            }
+            return null;
+        }
+
+        public void flatten(
+                List<RelNode> rels,
+                int systemFieldCount,
+                int[] start,
+                List<Pair<RelNode, Integer>> relOffsetList) {
+            for (RelNode rel : rels) {
+                if (leaves.containsKey(rel)) {
+                    relOffsetList.add(Pair.of(rel, start[0]));
+                    start[0] += leaves.get(rel);
+                } else if (rel instanceof LogicalMatch) {
+                    relOffsetList.add(Pair.of(rel, start[0]));
+                    start[0] += rel.getRowType().getFieldCount();
+                } else {
+                    if (rel instanceof LogicalJoin || rel instanceof LogicalAggregate) {
+                        start[0] += systemFieldCount;
+                    }
+                    flatten(rel.getInputs(), systemFieldCount, start, relOffsetList);
+                }
+            }
+        }
+
+        void registerSubQuery(SqlNode node, RelOptUtil.Logic logic) {
+            for (SubQuery subQuery : subQueryList) {
+                // Compare the reference to make sure the matched node has
+                // exact scope where it belongs.
+                if (node == subQuery.node) {
+                    return;
+                }
+            }
+            subQueryList.add(new SubQuery(node, logic));
+        }
+
+        SubQuery getSubQuery(SqlNode expr) {
+            for (SubQuery subQuery : subQueryList) {
+                // Compare the reference to make sure the matched node has
+                // exact scope where it belongs.
+                if (expr == subQuery.node) {
+                    return subQuery;
+                }
+            }
+
+            return null;
+        }
+
+        com.google.common.collect.ImmutableList<RelNode> retrieveCursors() {
+            try {
+                return com.google.common.collect.ImmutableList.copyOf(cursors);
+            } finally {
+                cursors.clear();
+            }
+        }
+
+        public RexNode convertExpression(SqlNode expr) {
+            // If we're in aggregation mode and this is an expression in the
+            // GROUP BY clause, return a reference to the field.
+            if (agg != null) {
+                final SqlNode expandedGroupExpr = validator.expand(expr, scope);
+                final int ref = agg.lookupGroupExpr(expandedGroupExpr);
+                if (ref >= 0) {
+                    return rexBuilder.makeInputRef(root, ref);
+                }
+                if (expr instanceof SqlCall) {
+                    final RexNode rex = agg.lookupAggregates((SqlCall) expr);
+                    if (rex != null) {
+                        return rex;
+                    }
+                }
+            }
+
+            // Allow the derived class chance to override the standard
+            // behavior for special kinds of expressions.
+            RexNode rex = convertExtendedExpression(expr, this);
+            if (rex != null) {
+                return rex;
+            }
+
+            // Sub-queries and OVER expressions are not like ordinary
+            // expressions.
+            final SqlKind kind = expr.getKind();
+            final SubQuery subQuery;
+            if (!config.isExpand()) {
+                final SqlCall call;
+                final SqlNode query;
+                final RelRoot root;
+                switch (kind) {
+                    case IN:
+                    case NOT_IN:
+                    case SOME:
+                    case ALL:
+                        call = (SqlCall) expr;
+                        query = call.operand(1);
+                        if (!(query instanceof SqlNodeList)) {
+                            root = convertQueryRecursive(query, false, null);
+                            final SqlNode operand = call.operand(0);
+                            List<SqlNode> nodes;
+                            switch (operand.getKind()) {
+                                case ROW:
+                                    nodes = ((SqlCall) operand).getOperandList();
+                                    break;
+                                default:
+                                    nodes = com.google.common.collect.ImmutableList.of(operand);
+                            }
+                            final com.google.common.collect.ImmutableList.Builder<RexNode> builder =
+                                    com.google.common.collect.ImmutableList.builder();
+                            for (SqlNode node : nodes) {
+                                builder.add(convertExpression(node));
+                            }
+                            final com.google.common.collect.ImmutableList<RexNode> list =
+                                    builder.build();
+                            switch (kind) {
+                                case IN:
+                                    return RexSubQuery.in(root.rel, list);
+                                case NOT_IN:
+                                    return rexBuilder.makeCall(
+                                            SqlStdOperatorTable.NOT,
+                                            RexSubQuery.in(root.rel, list));
+                                case SOME:
+                                    return RexSubQuery.some(
+                                            root.rel,
+                                            list,
+                                            (SqlQuantifyOperator) call.getOperator());
+                                case ALL:
+                                    return rexBuilder.makeCall(
+                                            SqlStdOperatorTable.NOT,
+                                            RexSubQuery.some(
+                                                    root.rel,
+                                                    list,
+                                                    negate(
+                                                            (SqlQuantifyOperator)
+                                                                    call.getOperator())));
+                                default:
+                                    throw new AssertionError(kind);
+                            }
+                        }
+                        break;
+
+                    case EXISTS:
+                        call = (SqlCall) expr;
+                        query =
+                                com.google.common.collect.Iterables.getOnlyElement(
+                                        call.getOperandList());
+                        root = convertQueryRecursive(query, false, null);
+                        RelNode rel = root.rel;
+                        while (rel instanceof Project
+                                || rel instanceof Sort
+                                        && ((Sort) rel).fetch == null
+                                        && ((Sort) rel).offset == null) {
+                            rel = ((SingleRel) rel).getInput();
+                        }
+                        return RexSubQuery.exists(rel);
+
+                    case SCALAR_QUERY:
+                        call = (SqlCall) expr;
+                        query =
+                                com.google.common.collect.Iterables.getOnlyElement(
+                                        call.getOperandList());
+                        root = convertQueryRecursive(query, false, null);
+                        return RexSubQuery.scalar(root.rel);
+                }
+            }
+
+            switch (kind) {
+                case SOME:
+                case ALL:
+                    if (config.isExpand()) {
+                        throw new RuntimeException(kind + " is only supported if expand = false");
+                    }
+                    // fall through
+                case CURSOR:
+                case IN:
+                case NOT_IN:
+                    subQuery = Objects.requireNonNull(getSubQuery(expr));
+                    rex = Objects.requireNonNull(subQuery.expr);
+                    return StandardConvertletTable.castToValidatedType(
+                            expr, rex, validator, rexBuilder);
+
+                case SELECT:
+                case EXISTS:
+                case SCALAR_QUERY:
+                    subQuery = getSubQuery(expr);
+                    assert subQuery != null;
+                    rex = subQuery.expr;
+                    assert rex != null : "rex != null";
+
+                    if (((kind == SqlKind.SCALAR_QUERY) || (kind == SqlKind.EXISTS))
+                            && isConvertedSubq(rex)) {
+                        // scalar sub-query or EXISTS has been converted to a
+                        // constant
+                        return rex;
+                    }
+
+                    // The indicator column is the last field of the sub-query.
+                    RexNode fieldAccess =
+                            rexBuilder.makeFieldAccess(rex, rex.getType().getFieldCount() - 1);
+
+                    // The indicator column will be nullable if it comes from
+                    // the null-generating side of the join. For EXISTS, add an
+                    // "IS TRUE" check so that the result is "BOOLEAN NOT NULL".
+                    if (fieldAccess.getType().isNullable() && kind == SqlKind.EXISTS) {
+                        fieldAccess =
+                                rexBuilder.makeCall(SqlStdOperatorTable.IS_NOT_NULL, fieldAccess);
+                    }
+                    return fieldAccess;
+
+                case OVER:
+                    return convertOver(this, expr);
+
+                default:
+                    // fall through
+            }
+
+            // Apply standard conversions.
+            rex = expr.accept(this);
+            return Objects.requireNonNull(rex);
+        }
+
+        /**
+         * Converts an item in an ORDER BY clause inside a window (OVER) clause, extracting DESC,
+         * NULLS LAST and NULLS FIRST flags first.
+         */
+        public RexFieldCollation convertSortExpression(
+                SqlNode expr,
+                RelFieldCollation.Direction direction,
+                RelFieldCollation.NullDirection nullDirection) {
+            switch (expr.getKind()) {
+                case DESCENDING:
+                    return convertSortExpression(
+                            ((SqlCall) expr).operand(0),
+                            RelFieldCollation.Direction.DESCENDING,
+                            nullDirection);
+                case NULLS_LAST:
+                    return convertSortExpression(
+                            ((SqlCall) expr).operand(0),
+                            direction,
+                            RelFieldCollation.NullDirection.LAST);
+                case NULLS_FIRST:
+                    return convertSortExpression(
+                            ((SqlCall) expr).operand(0),
+                            direction,
+                            RelFieldCollation.NullDirection.FIRST);
+                default:
+                    final Set<SqlKind> flags = EnumSet.noneOf(SqlKind.class);
+                    switch (direction) {
+                        case DESCENDING:
+                            flags.add(SqlKind.DESCENDING);
+                    }
+                    switch (nullDirection) {
+                        case UNSPECIFIED:
+                            final RelFieldCollation.NullDirection nullDefaultDirection =
+                                    validator.config().defaultNullCollation().last(desc(direction))
+                                            ? RelFieldCollation.NullDirection.LAST
+                                            : RelFieldCollation.NullDirection.FIRST;
+                            if (nullDefaultDirection != direction.defaultNullDirection()) {
+                                SqlKind nullDirectionSqlKind =
+                                        validator
+                                                        .config()
+                                                        .defaultNullCollation()
+                                                        .last(desc(direction))
+                                                ? SqlKind.NULLS_LAST
+                                                : SqlKind.NULLS_FIRST;
+                                flags.add(nullDirectionSqlKind);
+                            }
+                            break;
+                        case FIRST:
+                            flags.add(SqlKind.NULLS_FIRST);
+                            break;
+                        case LAST:
+                            flags.add(SqlKind.NULLS_LAST);
+                            break;
+                    }
+                    return new RexFieldCollation(convertExpression(expr), flags);
+            }
+        }
+
+        /**
+         * Determines whether a RexNode corresponds to a sub-query that's been converted to a
+         * constant.
+         *
+         * @param rex the expression to be examined
+         * @return true if the expression is a dynamic parameter, a literal, or a literal that is
+         *     being cast
+         */
+        private boolean isConvertedSubq(RexNode rex) {
+            if ((rex instanceof RexLiteral) || (rex instanceof RexDynamicParam)) {
+                return true;
+            }
+            if (rex instanceof RexCall) {
+                RexCall call = (RexCall) rex;
+                if (call.getOperator() == SqlStdOperatorTable.CAST) {
+                    RexNode operand = call.getOperands().get(0);
+                    if (operand instanceof RexLiteral) {
+                        return true;
+                    }
+                }
+            }
+            return false;
+        }
+
+        public int getGroupCount() {
+            if (agg != null) {
+                return agg.groupExprs.size();
+            }
+            if (window != null) {
+                return window.isAlwaysNonEmpty() ? 1 : 0;
+            }
+            return -1;
+        }
+
+        public RexBuilder getRexBuilder() {
+            return rexBuilder;
+        }
+
+        public SqlNode validateExpression(RelDataType rowType, SqlNode expr) {
+            return SqlValidatorUtil.validateExprWithRowType(
+                            catalogReader.nameMatcher().isCaseSensitive(),
+                            opTab,
+                            typeFactory,
+                            rowType,
+                            expr)
+                    .left;
+        }
+
+        public RexRangeRef getSubQueryExpr(SqlCall call) {
+            final SubQuery subQuery = getSubQuery(call);
+            assert subQuery != null;
+            return (RexRangeRef) subQuery.expr;
+        }
+
+        public RelDataTypeFactory getTypeFactory() {
+            return typeFactory;
+        }
+
+        public InitializerExpressionFactory getInitializerExpressionFactory() {
+            return initializerExpressionFactory;
+        }
+
+        public SqlValidator getValidator() {
+            return validator;
+        }
+
+        public RexNode convertLiteral(SqlLiteral literal) {
+            return exprConverter.convertLiteral(this, literal);
+        }
+
+        public RexNode convertInterval(SqlIntervalQualifier intervalQualifier) {
+            return exprConverter.convertInterval(this, intervalQualifier);
+        }
+
+        public RexNode visit(SqlLiteral literal) {
+            return exprConverter.convertLiteral(this, literal);
+        }
+
+        public RexNode visit(SqlCall call) {
+            if (agg != null) {
+                final SqlOperator op = call.getOperator();
+                if (window == null
+                        && (op.isAggregator()
+                                || op.getKind() == SqlKind.FILTER
+                                || op.getKind() == SqlKind.WITHIN_GROUP)) {
+                    return agg.lookupAggregates(call);
+                }
+            }
+            return exprConverter.convertCall(
+                    this, new SqlCallBinding(validator, scope, call).permutedCall());
+        }
+
+        public RexNode visit(SqlNodeList nodeList) {
+            throw new UnsupportedOperationException();
+        }
+
+        public RexNode visit(SqlIdentifier id) {
+            return convertIdentifier(this, id);
+        }
+
+        public RexNode visit(SqlDataTypeSpec type) {
+            throw new UnsupportedOperationException();
+        }
+
+        public RexNode visit(SqlDynamicParam param) {
+            return convertDynamicParam(param);
+        }
+
+        public RexNode visit(SqlIntervalQualifier intervalQualifier) {
+            return convertInterval(intervalQualifier);
+        }
+
+        public List<SqlMonotonicity> getColumnMonotonicities() {
+            return columnMonotonicities;
+        }
+    }
+
+    private SqlQuantifyOperator negate(SqlQuantifyOperator operator) {
+        assert operator.kind == SqlKind.ALL;
+        return SqlStdOperatorTable.some(operator.comparisonKind.negateNullSafe());
+    }
+
+    /** Deferred lookup. */
+    private static class DeferredLookup {
+        Blackboard bb;
+        String originalRelName;
+
+        DeferredLookup(Blackboard bb, String originalRelName) {
+            this.bb = bb;
+            this.originalRelName = originalRelName;
+        }
+
+        public RexFieldAccess getFieldAccess(CorrelationId name) {
+            return (RexFieldAccess) bb.mapCorrelateToRex.get(name);
+        }
+
+        public String getOriginalRelName() {
+            return originalRelName;
+        }
+    }
+
+    /** A default implementation of SubQueryConverter that does no conversion. */
+    private class NoOpSubQueryConverter implements SubQueryConverter {
+        public boolean canConvertSubQuery() {
+            return false;
+        }
+
+        public RexNode convertSubQuery(
+                SqlCall subQuery,
+                SqlToRelConverter parentConverter,
+                boolean isExists,
+                boolean isExplain) {
+            throw new IllegalArgumentException();
+        }
+    }
+
+    /**
+     * Converts expressions to aggregates.
+     *
+     * <p>Consider the expression
+     *
+     * <blockquote>
+     *
+     * {@code SELECT deptno, SUM(2 * sal) FROM emp GROUP BY deptno}
+     *
+     * </blockquote>
+     *
+     * <p>Then:
+     *
+     * <ul>
+     *   <li>groupExprs = {SqlIdentifier(deptno)}
+     *   <li>convertedInputExprs = {RexInputRef(deptno), 2 * RefInputRef(sal)}
+     *   <li>inputRefs = {RefInputRef(#0), RexInputRef(#1)}
+     *   <li>aggCalls = {AggCall(SUM, {1})}
+     * </ul>
+     */
+    protected class AggConverter implements SqlVisitor<Void> {
+        private final Blackboard bb;
+        public final AggregatingSelectScope aggregatingSelectScope;
+
+        private final Map<String, String> nameMap = new HashMap<>();
+
+        /** The group-by expressions, in {@link SqlNode} format. */
+        private final SqlNodeList groupExprs = new SqlNodeList(SqlParserPos.ZERO);
+
+        /** The auxiliary group-by expressions. */
+        private final Map<SqlNode, Ord<AuxiliaryConverter>> auxiliaryGroupExprs = new HashMap<>();
+
+        /**
+         * Input expressions for the group columns and aggregates, in {@link RexNode} format. The
+         * first elements of the list correspond to the elements in {@link #groupExprs}; the
+         * remaining elements are for aggregates. The right field of each pair is the name of the
+         * expression, where the expressions are simple mappings to input fields.
+         */
+        private final List<Pair<RexNode, String>> convertedInputExprs = new ArrayList<>();
+
+        /**
+         * Expressions to be evaluated as rows are being placed into the aggregate's hash table.
+         * This is when group functions such as TUMBLE cause rows to be expanded.
+         */
+        private final List<AggregateCall> aggCalls = new ArrayList<>();
+
+        private final Map<SqlNode, RexNode> aggMapping = new HashMap<>();
+        private final Map<AggregateCall, RexNode> aggCallMapping = new HashMap<>();
+
+        /** Whether we are directly inside a windowed aggregate. */
+        private boolean inOver = false;
+
+        AggConverter(Blackboard bb, AggregatingSelectScope aggregatingSelectScope) {
+            this.bb = bb;
+            this.aggregatingSelectScope = aggregatingSelectScope;
+        }
+
+        /**
+         * Creates an AggConverter.
+         *
+         * <p>The <code>select</code> parameter provides enough context to name aggregate calls
+         * which are top-level select list items.
+         *
+         * @param bb Blackboard
+         * @param select Query being translated; provides context to give
+         */
+        public AggConverter(Blackboard bb, SqlSelect select) {
+            this(bb, (AggregatingSelectScope) bb.getValidator().getSelectScope(select));
+
+            // Collect all expressions used in the select list so that aggregate
+            // calls can be named correctly.
+            final SqlNodeList selectList = select.getSelectList();
+            for (int i = 0; i < selectList.size(); i++) {
+                SqlNode selectItem = selectList.get(i);
+                String name = null;
+                if (SqlUtil.isCallTo(selectItem, SqlStdOperatorTable.AS)) {
+                    final SqlCall call = (SqlCall) selectItem;
+                    selectItem = call.operand(0);
+                    name = call.operand(1).toString();
+                }
+                if (name == null) {
+                    name = validator.deriveAlias(selectItem, i);
+                }
+                nameMap.put(selectItem.toString(), name);
+            }
+        }
+
+        public int addGroupExpr(SqlNode expr) {
+            int ref = lookupGroupExpr(expr);
+            if (ref >= 0) {
+                return ref;
+            }
+            final int index = groupExprs.size();
+            groupExprs.add(expr);
+            String name = nameMap.get(expr.toString());
+            RexNode convExpr = bb.convertExpression(expr);
+            addExpr(convExpr, name);
+
+            if (expr instanceof SqlCall) {
+                SqlCall call = (SqlCall) expr;
+                for (Pair<SqlNode, AuxiliaryConverter> p :
+                        SqlStdOperatorTable.convertGroupToAuxiliaryCalls(call)) {
+                    addAuxiliaryGroupExpr(p.left, index, p.right);
+                }
+            }
+
+            return index;
+        }
+
+        void addAuxiliaryGroupExpr(SqlNode node, int index, AuxiliaryConverter converter) {
+            for (SqlNode node2 : auxiliaryGroupExprs.keySet()) {
+                if (node2.equalsDeep(node, Litmus.IGNORE)) {
+                    return;
+                }
+            }
+            auxiliaryGroupExprs.put(node, Ord.of(index, converter));
+        }
+
+        /**
+         * Adds an expression, deducing an appropriate name if possible.
+         *
+         * @param expr Expression
+         * @param name Suggested name
+         */
+        private void addExpr(RexNode expr, String name) {
+            if ((name == null) && (expr instanceof RexInputRef)) {
+                final int i = ((RexInputRef) expr).getIndex();
+                name = bb.root.getRowType().getFieldList().get(i).getName();
+            }
+            if (Pair.right(convertedInputExprs).contains(name)) {
+                // In case like 'SELECT ... GROUP BY x, y, x', don't add
+                // name 'x' twice.
+                name = null;
+            }
+            convertedInputExprs.add(Pair.of(expr, name));
+        }
+
+        public Void visit(SqlIdentifier id) {
+            return null;
+        }
+
+        public Void visit(SqlNodeList nodeList) {
+            for (int i = 0; i < nodeList.size(); i++) {
+                nodeList.get(i).accept(this);
+            }
+            return null;
+        }
+
+        public Void visit(SqlLiteral lit) {
+            return null;
+        }
+
+        public Void visit(SqlDataTypeSpec type) {
+            return null;
+        }
+
+        public Void visit(SqlDynamicParam param) {
+            return null;
+        }
+
+        public Void visit(SqlIntervalQualifier intervalQualifier) {
+            return null;
+        }
+
+        public Void visit(SqlCall call) {
+            switch (call.getKind()) {
+                case FILTER:
+                case WITHIN_GROUP:
+                    translateAgg(call);
+                    return null;
+                case SELECT:
+                    // rchen 2006-10-17:
+                    // for now do not detect aggregates in sub-queries.
+                    return null;
+            }
+            final boolean prevInOver = inOver;
+            // Ignore window aggregates and ranking functions (associated with OVER
+            // operator). However, do not ignore nested window aggregates.
+            if (call.getOperator().getKind() == SqlKind.OVER) {
+                // Track aggregate nesting levels only within an OVER operator.
+                List<SqlNode> operandList = call.getOperandList();
+                assert operandList.size() == 2;
+
+                // Ignore the top level window aggregates and ranking functions
+                // positioned as the first operand of a OVER operator
+                inOver = true;
+                operandList.get(0).accept(this);
+
+                // Normal translation for the second operand of a OVER operator
+                inOver = false;
+                operandList.get(1).accept(this);
+                return null;
+            }
+
+            // Do not translate the top level window aggregate. Only do so for
+            // nested aggregates, if present
+            if (call.getOperator().isAggregator()) {
+                if (inOver) {
+                    // Add the parent aggregate level before visiting its children
+                    inOver = false;
+                } else {
+                    // We're beyond the one ignored level
+                    translateAgg(call);
+                    return null;
+                }
+            }
+            for (SqlNode operand : call.getOperandList()) {
+                // Operands are occasionally null, e.g. switched CASE arg 0.
+                if (operand != null) {
+                    operand.accept(this);
+                }
+            }
+            // Remove the parent aggregate level after visiting its children
+            inOver = prevInOver;
+            return null;
+        }
+
+        private void translateAgg(SqlCall call) {
+            translateAgg(call, null, null, false, call);
+        }
+
+        private void translateAgg(
+                SqlCall call,
+                SqlNode filter,
+                SqlNodeList orderList,
+                boolean ignoreNulls,
+                SqlCall outerCall) {
+            assert bb.agg == this;
+            assert outerCall != null;
+            switch (call.getKind()) {
+                case FILTER:
+                    assert filter == null;
+                    translateAgg(
+                            call.operand(0), call.operand(1), orderList, ignoreNulls, outerCall);
+                    return;
+                case WITHIN_GROUP:
+                    assert orderList == null;
+                    translateAgg(call.operand(0), filter, call.operand(1), ignoreNulls, outerCall);
+                    return;
+                case IGNORE_NULLS:
+                    ignoreNulls = true;
+                    // fall through
+                case RESPECT_NULLS:
+                    translateAgg(call.operand(0), filter, orderList, ignoreNulls, outerCall);
+                    return;
+            }
+            final List<Integer> args = new ArrayList<>();
+            int filterArg = -1;
+            final List<RelDataType> argTypes =
+                    call.getOperator() instanceof SqlCountAggFunction
+                            ? new ArrayList<>(call.getOperandList().size())
+                            : null;
+            try {
+                // switch out of agg mode
+                bb.agg = null;
+                for (SqlNode operand : call.getOperandList()) {
+
+                    // special case for COUNT(*):  delete the *
+                    if (operand instanceof SqlIdentifier) {
+                        SqlIdentifier id = (SqlIdentifier) operand;
+                        if (id.isStar()) {
+                            assert call.operandCount() == 1;
+                            assert args.isEmpty();
+                            break;
+                        }
+                    }
+                    RexNode convertedExpr = bb.convertExpression(operand);
+                    assert convertedExpr != null;
+                    if (argTypes != null) {
+                        argTypes.add(convertedExpr.getType());
+                    }
+                    args.add(lookupOrCreateGroupExpr(convertedExpr));
+                }
+
+                if (filter != null) {
+                    RexNode convertedExpr = bb.convertExpression(filter);
+                    assert convertedExpr != null;
+                    if (convertedExpr.getType().isNullable()) {
+                        convertedExpr =
+                                rexBuilder.makeCall(SqlStdOperatorTable.IS_TRUE, convertedExpr);
+                    }
+                    filterArg = lookupOrCreateGroupExpr(convertedExpr);
+                }
+            } finally {
+                // switch back into agg mode
+                bb.agg = this;
+            }
+
+            SqlAggFunction aggFunction = (SqlAggFunction) call.getOperator();
+            final RelDataType type = validator.deriveType(bb.scope, call);
+            boolean distinct = false;
+            SqlLiteral quantifier = call.getFunctionQuantifier();
+            if ((null != quantifier) && (quantifier.getValue() == SqlSelectKeyword.DISTINCT)) {
+                distinct = true;
+            }
+            boolean approximate = false;
+            if (aggFunction == SqlStdOperatorTable.APPROX_COUNT_DISTINCT) {
+                aggFunction = SqlStdOperatorTable.COUNT;
+                distinct = true;
+                approximate = true;
+            }
+            final RelCollation collation;
+            if (orderList == null || orderList.size() == 0) {
+                collation = RelCollations.EMPTY;
+            } else {
+                collation =
+                        RelCollations.of(
+                                orderList.getList().stream()
+                                        .map(
+                                                order ->
+                                                        bb.convertSortExpression(
+                                                                order,
+                                                                RelFieldCollation.Direction
+                                                                        .ASCENDING,
+                                                                RelFieldCollation.NullDirection
+                                                                        .UNSPECIFIED))
+                                        .map(
+                                                fieldCollation ->
+                                                        new RelFieldCollation(
+                                                                lookupOrCreateGroupExpr(
+                                                                        fieldCollation.left),
+                                                                fieldCollation.getDirection(),
+                                                                fieldCollation.getNullDirection()))
+                                        .collect(Collectors.toList()));
+            }
+            final AggregateCall aggCall =
+                    AggregateCall.create(
+                            aggFunction,
+                            distinct,
+                            approximate,
+                            ignoreNulls,
+                            args,
+                            filterArg,
+                            collation,
+                            type,
+                            nameMap.get(outerCall.toString()));
+            RexNode rex =
+                    rexBuilder.addAggCall(
+                            aggCall, groupExprs.size(), aggCalls, aggCallMapping, argTypes);
+            aggMapping.put(outerCall, rex);
+        }
+
+        private int lookupOrCreateGroupExpr(RexNode expr) {
+            int index = 0;
+            for (RexNode convertedInputExpr : Pair.left(convertedInputExprs)) {
+                if (expr.equals(convertedInputExpr)) {
+                    return index;
+                }
+                ++index;
+            }
+
+            // not found -- add it
+            addExpr(expr, null);
+            return index;
+        }
+
+        /**
+         * If an expression is structurally identical to one of the group-by expressions, returns a
+         * reference to the expression, otherwise returns null.
+         */
+        public int lookupGroupExpr(SqlNode expr) {
+            for (int i = 0; i < groupExprs.size(); i++) {
+                SqlNode groupExpr = groupExprs.get(i);
+                if (expr.equalsDeep(groupExpr, Litmus.IGNORE)) {
+                    return i;
+                }
+            }
+            return -1;
+        }
+
+        public RexNode lookupAggregates(SqlCall call) {
+            // assert call.getOperator().isAggregator();
+            assert bb.agg == this;
+
+            for (Map.Entry<SqlNode, Ord<AuxiliaryConverter>> e : auxiliaryGroupExprs.entrySet()) {
+                if (call.equalsDeep(e.getKey(), Litmus.IGNORE)) {
+                    AuxiliaryConverter converter = e.getValue().e;
+                    final int groupOrdinal = e.getValue().i;
+                    return converter.convert(
+                            rexBuilder,
+                            convertedInputExprs.get(groupOrdinal).left,
+                            rexBuilder.makeInputRef(bb.root, groupOrdinal));
+                }
+            }
+
+            return aggMapping.get(call);
+        }
+
+        public List<Pair<RexNode, String>> getPreExprs() {
+            return convertedInputExprs;
+        }
+
+        public List<AggregateCall> getAggCalls() {
+            return aggCalls;
+        }
+
+        public RelDataTypeFactory getTypeFactory() {
+            return typeFactory;
+        }
+    }
+
+    /** Context to find a relational expression to a field offset. */
+    private static class LookupContext {
+        private final List<Pair<RelNode, Integer>> relOffsetList = new ArrayList<>();
+
+        /**
+         * Creates a LookupContext with multiple input relational expressions.
+         *
+         * @param bb Context for translating this sub-query
+         * @param rels Relational expressions
+         * @param systemFieldCount Number of system fields
+         */
+        LookupContext(Blackboard bb, List<RelNode> rels, int systemFieldCount) {
+            bb.flatten(rels, systemFieldCount, new int[] {0}, relOffsetList);
+        }
+
+        /**
+         * Returns the relational expression with a given offset, and the ordinal in the combined
+         * row of its first field.
+         *
+         * <p>For example, in {@code Emp JOIN Dept}, findRel(1) returns the relational expression
+         * for {@code Dept} and offset 6 (because {@code Emp} has 6 fields, therefore the first
+         * field of {@code Dept} is field 6.
+         *
+         * @param offset Offset of relational expression in FROM clause
+         * @return Relational expression and the ordinal of its first field
+         */
+        Pair<RelNode, Integer> findRel(int offset) {
+            return relOffsetList.get(offset);
+        }
+    }
+
+    /**
+     * Shuttle which walks over a tree of {@link RexNode}s and applies 'over' to all agg functions.
+     *
+     * <p>This is necessary because the returned expression is not necessarily a call to an agg
+     * function. For example,
+     *
+     * <blockquote>
+     *
+     * <code>AVG(x)</code>
+     *
+     * </blockquote>
+     *
+     * <p>becomes
+     *
+     * <blockquote>
+     *
+     * <code>SUM(x) / COUNT(x)</code>
+     *
+     * </blockquote>
+     *
+     * <p>Any aggregate functions are converted to calls to the internal <code>
+     * $Histogram</code> aggregation function and accessors such as <code>
+     * $HistogramMin</code>; for example,
+     *
+     * <blockquote>
+     *
+     * <code>MIN(x), MAX(x)</code>
+     *
+     * </blockquote>
+     *
+     * <p>are converted to
+     *
+     * <blockquote>
+     *
+     * <code>$HistogramMin($Histogram(x)),
+     * $HistogramMax($Histogram(x))</code>
+     *
+     * </blockquote>
+     *
+     * <p>Common sub-expression elimination will ensure that only one histogram is computed.
+     */
+    private class HistogramShuttle extends RexShuttle {
+        /**
+         * Whether to convert calls to MIN(x) to HISTOGRAM_MIN(HISTOGRAM(x)). Histograms allow
+         * rolling computation, but require more space.
+         */
+        static final boolean ENABLE_HISTOGRAM_AGG = false;
+
+        private final List<RexNode> partitionKeys;
+        private final com.google.common.collect.ImmutableList<RexFieldCollation> orderKeys;
+        private final RexWindowBound lowerBound;
+        private final RexWindowBound upperBound;
+        private final boolean rows;
+        private final boolean allowPartial;
+        private final boolean distinct;
+        private final boolean ignoreNulls;
+
+        HistogramShuttle(
+                List<RexNode> partitionKeys,
+                com.google.common.collect.ImmutableList<RexFieldCollation> orderKeys,
+                RexWindowBound lowerBound,
+                RexWindowBound upperBound,
+                boolean rows,
+                boolean allowPartial,
+                boolean distinct,
+                boolean ignoreNulls) {
+            this.partitionKeys = partitionKeys;
+            this.orderKeys = orderKeys;
+            this.lowerBound = lowerBound;
+            this.upperBound = upperBound;
+            this.rows = rows;
+            this.allowPartial = allowPartial;
+            this.distinct = distinct;
+            this.ignoreNulls = ignoreNulls;
+        }
+
+        public RexNode visitCall(RexCall call) {
+            final SqlOperator op = call.getOperator();
+            if (!(op instanceof SqlAggFunction)) {
+                return super.visitCall(call);
+            }
+            final SqlAggFunction aggOp = (SqlAggFunction) op;
+            final RelDataType type = call.getType();
+            List<RexNode> exprs = call.getOperands();
+
+            SqlFunction histogramOp = !ENABLE_HISTOGRAM_AGG ? null : getHistogramOp(aggOp);
+
+            if (histogramOp != null) {
+                final RelDataType histogramType = computeHistogramType(type);
+
+                // For DECIMAL, since it's already represented as a bigint we
+                // want to do a reinterpretCast instead of a cast to avoid
+                // losing any precision.
+                boolean reinterpretCast = type.getSqlTypeName() == SqlTypeName.DECIMAL;
+
+                // Replace original expression with CAST of not one
+                // of the supported types
+                if (histogramType != type) {
+                    exprs = new ArrayList<>(exprs);
+                    exprs.set(
+                            0,
+                            reinterpretCast
+                                    ? rexBuilder.makeReinterpretCast(
+                                            histogramType,
+                                            exprs.get(0),
+                                            rexBuilder.makeLiteral(false))
+                                    : rexBuilder.makeCast(histogramType, exprs.get(0)));
+                }
+
+                RexCallBinding bind =
+                        new RexCallBinding(
+                                rexBuilder.getTypeFactory(),
+                                SqlStdOperatorTable.HISTOGRAM_AGG,
+                                exprs,
+                                com.google.common.collect.ImmutableList.of());
+
+                RexNode over =
+                        rexBuilder.makeOver(
+                                SqlStdOperatorTable.HISTOGRAM_AGG.inferReturnType(bind),
+                                SqlStdOperatorTable.HISTOGRAM_AGG,
+                                exprs,
+                                partitionKeys,
+                                orderKeys,
+                                lowerBound,
+                                upperBound,
+                                rows,
+                                allowPartial,
+                                false,
+                                distinct,
+                                ignoreNulls);
+
+                RexNode histogramCall =
+                        rexBuilder.makeCall(
+                                histogramType,
+                                histogramOp,
+                                com.google.common.collect.ImmutableList.of(over));
+
+                // If needed, post Cast result back to original
+                // type.
+                if (histogramType != type) {
+                    if (reinterpretCast) {
+                        histogramCall =
+                                rexBuilder.makeReinterpretCast(
+                                        type, histogramCall, rexBuilder.makeLiteral(false));
+                    } else {
+                        histogramCall = rexBuilder.makeCast(type, histogramCall);
+                    }
+                }
+
+                return histogramCall;
+            } else {
+                boolean needSum0 = aggOp == SqlStdOperatorTable.SUM && type.isNullable();
+                SqlAggFunction aggOpToUse = needSum0 ? SqlStdOperatorTable.SUM0 : aggOp;
+                return rexBuilder.makeOver(
+                        type,
+                        aggOpToUse,
+                        exprs,
+                        partitionKeys,
+                        orderKeys,
+                        lowerBound,
+                        upperBound,
+                        rows,
+                        allowPartial,
+                        needSum0,
+                        distinct,
+                        ignoreNulls);
+            }
+        }
+
+        /**
+         * Returns the histogram operator corresponding to a given aggregate function.
+         *
+         * <p>For example, <code>getHistogramOp
+         * ({@link SqlStdOperatorTable#MIN}}</code> returns {@link
+         * SqlStdOperatorTable#HISTOGRAM_MIN}.
+         *
+         * @param aggFunction An aggregate function
+         * @return Its histogram function, or null
+         */
+        SqlFunction getHistogramOp(SqlAggFunction aggFunction) {
+            if (aggFunction == SqlStdOperatorTable.MIN) {
+                return SqlStdOperatorTable.HISTOGRAM_MIN;
+            } else if (aggFunction == SqlStdOperatorTable.MAX) {
+                return SqlStdOperatorTable.HISTOGRAM_MAX;
+            } else if (aggFunction == SqlStdOperatorTable.FIRST_VALUE) {
+                return SqlStdOperatorTable.HISTOGRAM_FIRST_VALUE;
+            } else if (aggFunction == SqlStdOperatorTable.LAST_VALUE) {
+                return SqlStdOperatorTable.HISTOGRAM_LAST_VALUE;
+            } else {
+                return null;
+            }
+        }
+
+        /**
+         * Returns the type for a histogram function. It is either the actual type or an an
+         * approximation to it.
+         */
+        private RelDataType computeHistogramType(RelDataType type) {
+            if (SqlTypeUtil.isExactNumeric(type) && type.getSqlTypeName() != SqlTypeName.BIGINT) {
+                return typeFactory.createSqlType(SqlTypeName.BIGINT);
+            } else if (SqlTypeUtil.isApproximateNumeric(type)
+                    && type.getSqlTypeName() != SqlTypeName.DOUBLE) {
+                return typeFactory.createSqlType(SqlTypeName.DOUBLE);
+            } else {
+                return type;
+            }
+        }
+    }
+
+    /** A sub-query, whether it needs to be translated using 2- or 3-valued logic. */
+    private static class SubQuery {
+        final SqlNode node;
+        final RelOptUtil.Logic logic;
+        RexNode expr;
+
+        private SubQuery(SqlNode node, RelOptUtil.Logic logic) {
+            this.node = node;
+            this.logic = logic;
+        }
+    }
+
+    /** Visitor that collects all aggregate functions in a {@link SqlNode} tree. */
+    private static class AggregateFinder extends SqlBasicVisitor<Void> {
+        final SqlNodeList list = new SqlNodeList(SqlParserPos.ZERO);
+        final SqlNodeList filterList = new SqlNodeList(SqlParserPos.ZERO);
+        final SqlNodeList orderList = new SqlNodeList(SqlParserPos.ZERO);
+
+        @Override
+        public Void visit(SqlCall call) {
+            // ignore window aggregates and ranking functions (associated with OVER operator)
+            if (call.getOperator().getKind() == SqlKind.OVER) {
+                return null;
+            }
+
+            if (call.getOperator().getKind() == SqlKind.FILTER) {
+                // the WHERE in a FILTER must be tracked too so we can call replaceSubQueries on it.
+                // see https://issues.apache.org/jira/browse/CALCITE-1910
+                final SqlNode aggCall = call.getOperandList().get(0);
+                final SqlNode whereCall = call.getOperandList().get(1);
+                list.add(aggCall);
+                filterList.add(whereCall);
+                return null;
+            }
+
+            if (call.getOperator().getKind() == SqlKind.WITHIN_GROUP) {
+                // the WHERE in a WITHIN_GROUP must be tracked too so we can call replaceSubQueries
+                // on it.
+                // see https://issues.apache.org/jira/browse/CALCITE-1910
+                final SqlNode aggCall = call.getOperandList().get(0);
+                final SqlNodeList orderList = (SqlNodeList) call.getOperandList().get(1);
+                list.add(aggCall);
+                orderList.getList().forEach(this.orderList::add);
+                return null;
+            }
+
+            if (call.getOperator().isAggregator()) {
+                list.add(call);
+                return null;
+            }
+
+            // Don't traverse into sub-queries, even if they contain aggregate
+            // functions.
+            if (call instanceof SqlSelect) {
+                return null;
+            }
+
+            return call.getOperator().acceptCall(this, call);
+        }
+    }
+
+    /** Use of a row as a correlating variable by a given relational expression. */
+    private static class CorrelationUse {
+        private final CorrelationId id;
+        private final ImmutableBitSet requiredColumns;
+        /** The relational expression that uses the variable. */
+        private final RelNode r;
+
+        CorrelationUse(CorrelationId id, ImmutableBitSet requiredColumns, RelNode r) {
+            this.id = id;
+            this.requiredColumns = requiredColumns;
+            this.r = r;
+        }
+    }
+
+    /** Creates a builder for a {@link Config}. */
+    @Deprecated // to be removed before 2.0
+    public static ConfigBuilder configBuilder() {
+        return new ConfigBuilder();
+    }
+
+    /** Returns a default {@link Config}. */
+    public static Config config() {
+        return CONFIG;
+    }
+
+    /**
+     * Interface to define the configuration for a SqlToRelConverter. Provides methods to set each
+     * configuration option.
+     *
+     * @see SqlToRelConverter#CONFIG
+     */
+    public interface Config {
+        /**
+         * Returns the {@code decorrelationEnabled} option. Controls whether to disable sub-query
+         * decorrelation when needed. e.g. if outer joins are not supported.
+         */
+        @ImmutableBeans.Property
+        @ImmutableBeans.BooleanDefault(true)
+        boolean isDecorrelationEnabled();
+
+        /** Sets {@link #isDecorrelationEnabled()}. */
+        Config withDecorrelationEnabled(boolean decorrelationEnabled);
+
+        /**
+         * Returns the {@code trimUnusedFields} option. Controls whether to trim unused fields as
+         * part of the conversion process.
+         */
+        @ImmutableBeans.Property
+        @ImmutableBeans.BooleanDefault(false)
+        boolean isTrimUnusedFields();
+
+        /** Sets {@link #isTrimUnusedFields()}. */
+        Config withTrimUnusedFields(boolean trimUnusedFields);
+
+        /**
+         * Returns the {@code createValuesRel} option. Controls whether instances of {@link
+         * LogicalValues} are generated. These may not be supported by all physical implementations.
+         */
+        @ImmutableBeans.Property
+        @ImmutableBeans.BooleanDefault(true)
+        boolean isCreateValuesRel();
+
+        /** Sets {@link #isCreateValuesRel()}. */
+        Config withCreateValuesRel(boolean createValuesRel);
+
+        /**
+         * Returns the {@code explain} option. Describes whether the current statement is part of an
+         * EXPLAIN PLAN statement.
+         */
+        @ImmutableBeans.Property
+        @ImmutableBeans.BooleanDefault(false)
+        boolean isExplain();
+
+        /** Sets {@link #isExplain()}. */
+        Config withExplain(boolean explain);
+
+        /**
+         * Returns the {@code expand} option. Controls whether to expand sub-queries. If false, each
+         * sub-query becomes a {@link RexSubQuery}.
+         */
+        @ImmutableBeans.Property
+        @ImmutableBeans.BooleanDefault(true)
+        boolean isExpand();
+
+        /** Sets {@link #isExpand()}. */
+        Config withExpand(boolean expand);
+
+        /**
+         * Returns the {@code inSubQueryThreshold} option, default {@link
+         * #DEFAULT_IN_SUB_QUERY_THRESHOLD}. Controls the list size threshold under which {@link
+         * #convertInToOr} is used. Lists of this size or greater will instead be converted to use a
+         * join against an inline table ({@link LogicalValues}) rather than a predicate. A threshold
+         * of 0 forces usage of an inline table in all cases; a threshold of {@link
+         * Integer#MAX_VALUE} forces usage of OR in all cases.
+         */
+        @ImmutableBeans.Property
+        @ImmutableBeans.IntDefault(DEFAULT_IN_SUB_QUERY_THRESHOLD)
+        int getInSubQueryThreshold();
+
+        /** Sets {@link #getInSubQueryThreshold()}. */
+        Config withInSubQueryThreshold(int threshold);
+
+        /**
+         * Returns whether to remove Sort operator for a sub-query if the Sort has no offset and
+         * fetch limit attributes. Because the remove does not change the semantics, in many cases
+         * this is a promotion. Default is true.
+         */
+        @ImmutableBeans.Property
+        @ImmutableBeans.BooleanDefault(true)
+        boolean isRemoveSortInSubQuery();
+
+        /** Sets {@link #isRemoveSortInSubQuery()}. */
+        Config withRemoveSortInSubQuery(boolean removeSortInSubQuery);
+
+        /**
+         * Returns the factory to create {@link RelBuilder}, never null. Default is {@link
+         * RelFactories#LOGICAL_BUILDER}.
+         */
+        @ImmutableBeans.Property(required = true)
+        RelBuilderFactory getRelBuilderFactory();
+
+        /** Sets {@link #getRelBuilderFactory()}. */
+        Config withRelBuilderFactory(RelBuilderFactory factory);
+
+        /**
+         * Returns a function that takes a {@link RelBuilder.Config} and returns another. Default is
+         * the identity function.
+         */
+        @ImmutableBeans.Property(required = true)
+        UnaryOperator<RelBuilder.Config> getRelBuilderConfigTransform();
+
+        /**
+         * Sets {@link #getRelBuilderConfigTransform()}.
+         *
+         * @see #addRelBuilderConfigTransform
+         */
+        Config withRelBuilderConfigTransform(UnaryOperator<RelBuilder.Config> transform);
+
+        /** Adds a transform to {@link #getRelBuilderConfigTransform()}. */
+        default Config addRelBuilderConfigTransform(UnaryOperator<RelBuilder.Config> transform) {
+            return withRelBuilderConfigTransform(
+                    getRelBuilderConfigTransform().andThen(transform)::apply);
+        }
+
+        /**
+         * Returns the hint strategies used to decide how the hints are propagated to the relational
+         * expressions. Default is {@link HintStrategyTable#EMPTY}.
+         */
+        @ImmutableBeans.Property(required = true)
+        HintStrategyTable getHintStrategyTable();
+
+        /** Sets {@link #getHintStrategyTable()}. */
+        Config withHintStrategyTable(HintStrategyTable hintStrategyTable);
+    }
+
+    /** Builder for a {@link Config}. */
+    @Deprecated // to be removed before 2.0
+    public static class ConfigBuilder {
+        private Config config;
+
+        private ConfigBuilder() {
+            config = CONFIG;
+        }
+
+        /** Sets configuration identical to a given {@link Config}. */
+        public ConfigBuilder withConfig(Config config) {
+            this.config = config;
+            return this;
+        }
+
+        public ConfigBuilder withDecorrelationEnabled(boolean enabled) {
+            return withConfig(config.withDecorrelationEnabled(enabled));
+        }
+
+        public ConfigBuilder withTrimUnusedFields(boolean trimUnusedFields) {
+            return withConfig(config.withTrimUnusedFields(trimUnusedFields));
+        }
+
+        public ConfigBuilder withCreateValuesRel(boolean createValuesRel) {
+            return withConfig(config.withCreateValuesRel(createValuesRel));
+        }
+
+        public ConfigBuilder withExplain(boolean explain) {
+            return withConfig(config.withExplain(explain));
+        }
+
+        public ConfigBuilder withExpand(boolean expand) {
+            return withConfig(config.withExpand(expand));
+        }
+
+        public ConfigBuilder withRemoveSortInSubQuery(boolean removeSortInSubQuery) {
+            return withConfig(config.withRemoveSortInSubQuery(removeSortInSubQuery));
+        }
+
+        /** Whether to push down join conditions; default true. */
+        public ConfigBuilder withPushJoinCondition(boolean pushJoinCondition) {
+            return withRelBuilderConfigTransform(c -> c.withPushJoinCondition(pushJoinCondition));
+        }
+
+        public ConfigBuilder withInSubqueryThreshold(int inSubQueryThreshold) {
+            return withInSubQueryThreshold(inSubQueryThreshold);
+        }
+
+        public ConfigBuilder withInSubQueryThreshold(int inSubQueryThreshold) {
+            return withConfig(config.withInSubQueryThreshold(inSubQueryThreshold));
+        }
+
+        public ConfigBuilder withRelBuilderConfigTransform(
+                UnaryOperator<RelBuilder.Config> configTransform) {
+            return withConfig(config.addRelBuilderConfigTransform(configTransform));
+        }
+
+        public ConfigBuilder withRelBuilderFactory(RelBuilderFactory relBuilderFactory) {
+            return withConfig(config.withRelBuilderFactory(relBuilderFactory));
+        }
+
+        public ConfigBuilder withHintStrategyTable(HintStrategyTable hintStrategyTable) {
+            return withConfig(config.withHintStrategyTable(hintStrategyTable));
+        }
+
+        /** Builds a {@link Config}. */
+        public Config build() {
+            return config;
+        }
+    }
+}


[flink] 04/04: [FLINK-28682][table-planner] Support join hint in batch rules

Posted by go...@apache.org.
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

commit f794d72903c9c76e89731273dbb60bbee75e3166
Author: xuyang <xy...@163.com>
AuthorDate: Tue Jul 26 14:47:27 2022 +0800

    [FLINK-28682][table-planner] Support join hint in batch rules
    
    This closes #20359
---
 .../apache/calcite/sql2rel/RelDecorrelator.java    |   11 +
 .../apache/calcite/sql2rel/SqlToRelConverter.java  |   65 +-
 ...ClearJoinHintWithInvalidPropagationShuttle.java |  179 +++
 .../planner/calcite/RelTimeIndicatorConverter.java |    3 +-
 .../table/planner/hint/FlinkHintStrategies.java    |   54 +
 .../flink/table/planner/hint/FlinkHints.java       |  101 ++
 .../flink/table/planner/hint/JoinStrategy.java     |   87 ++
 .../optimize/ClearQueryBlockAliasResolver.java     |   71 ++
 .../planner/plan/optimize/JoinHintResolver.java    |  237 ++++
 .../StreamNonDeterministicUpdatePlanVisitor.java   |    9 +-
 .../planner/calcite/FlinkLogicalRelFactories.scala |    2 +-
 .../table/planner/calcite/FlinkPlannerImpl.scala   |   49 +-
 .../table/planner/plan/nodes/calcite/Sink.scala    |    2 +-
 .../plan/nodes/logical/FlinkLogicalJoin.scala      |   21 +-
 .../BatchCommonSubGraphBasedOptimizer.scala        |   15 +-
 .../logical/SplitPythonConditionFromJoinRule.scala |    1 +
 .../TemporalJoinRewriteWithUniqueKeyRule.scala     |    7 +-
 .../physical/batch/BatchPhysicalHashJoinRule.scala |  111 +-
 .../physical/batch/BatchPhysicalJoinRuleBase.scala |  258 +++-
 .../batch/BatchPhysicalNestedLoopJoinRule.scala    |   53 +-
 .../batch/BatchPhysicalSingleRowJoinRule.scala     |   23 +-
 .../batch/BatchPhysicalSortMergeJoinRule.scala     |   12 +-
 .../table/planner/plan/utils/FlinkRelOptUtil.scala |   10 +-
 .../planner/plan/utils/RelTreeWriterImpl.scala     |   36 +-
 ...rJoinHintWithInvalidPropagationShuttleTest.java |  225 ++++
 .../sql/join/joinhint/BroadcastJoinHintTest.java   |   35 +
 .../batch/sql/join/joinhint/JoinHintTestBase.java  |  726 ++++++++++++
 .../sql/join/joinhint/NestLoopJoinHintTest.java    |   35 +
 .../sql/join/joinhint/ShuffleHashJoinHintTest.java |   35 +
 .../join/joinhint/ShuffleMergeJoinHintTest.java    |   35 +
 .../optimize/ClearQueryBlockAliasResolverTest.java |  104 ++
 .../plan/optimize/JoinHintResolverTest.java        |  101 ++
 ...arJoinHintWithInvalidPropagationShuttleTest.xml |  150 +++
 .../plan/batch/sql/join/ShuffledHashJoinTest.xml   |   32 +
 .../sql/join/joinhint/BroadcastJoinHintTest.xml    | 1210 +++++++++++++++++++
 .../sql/join/joinhint/NestLoopJoinHintTest.xml     | 1206 +++++++++++++++++++
 .../sql/join/joinhint/ShuffleHashJoinHintTest.xml  | 1246 ++++++++++++++++++++
 .../sql/join/joinhint/ShuffleMergeJoinHintTest.xml | 1246 ++++++++++++++++++++
 .../optimize/ClearQueryBlockAliasResolverTest.xml  |  702 +++++++++++
 .../planner/plan/optimize/JoinHintResolverTest.xml |  702 +++++++++++
 .../plan/batch/sql/join/ShuffledHashJoinTest.scala |    8 -
 .../rules/logical/subquery/SubQueryTestBase.scala  |    4 +-
 .../flink/table/planner/utils/TableTestBase.scala  |   72 +-
 43 files changed, 9123 insertions(+), 168 deletions(-)

diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
index 07893cc4ad7..0d132041278 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.sql2rel;
 
+import org.apache.flink.table.planner.alias.ClearJoinHintWithInvalidPropagationShuttle;
+
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
@@ -203,6 +205,15 @@ public class RelDecorrelator implements ReflectiveVisitor {
         // Re-propagate the hints.
         newRootRel = RelOptUtil.propagateRelHints(newRootRel, true);
 
+        // ----- FLINK MODIFICATION BEGIN -----
+
+        // clear join hints which are propagated into wrong query block
+        // The hint QueryBlockAlias will be added when building a RelNode tree before. It is used to
+        // distinguish the query block in the SQL.
+        newRootRel = newRootRel.accept(new ClearJoinHintWithInvalidPropagationShuttle());
+
+        // ----- FLINK MODIFICATION END -----
+
         return newRootRel;
     }
 
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 11414792b74..41b1f266472 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -18,8 +18,8 @@
 
 package org.apache.calcite.sql2rel;
 
-import org.apache.flink.table.planner.alias.SubQueryAliasNodeClearShuttle;
-import org.apache.flink.table.planner.plan.nodes.calcite.LogicalSubQueryAlias;
+import org.apache.flink.table.planner.alias.ClearJoinHintWithInvalidPropagationShuttle;
+import org.apache.flink.table.planner.hint.FlinkHints;
 
 import org.apache.calcite.avatica.util.Spaces;
 import org.apache.calcite.linq4j.Ord;
@@ -598,8 +598,18 @@ public class SqlToRelConverter {
                 hints = SqlUtil.getRelHint(hintStrategies, select.getHints());
             }
         }
-        // propagate the hints.
+
         result = RelOptUtil.propagateRelHints(result, false);
+
+        // ----- FLINK MODIFICATION BEGIN -----
+
+        // clear join hints which are propagated into wrong query block
+        // The hint QueryBlockAlias will be added when building a RelNode tree before. It is used to
+        // distinguish the query block in the SQL.
+        result = result.accept(new ClearJoinHintWithInvalidPropagationShuttle());
+
+        // ----- FLINK MODIFICATION END -----
+
         return RelRoot.of(result, validatedRowType, query.getKind())
                 .withCollation(collation)
                 .withHints(hints);
@@ -2025,6 +2035,25 @@ public class SqlToRelConverter {
         convertFrom(bb, from, Collections.emptyList());
     }
 
+    // ----- FLINK MODIFICATION BEGIN -----
+
+    private boolean containsJoinHint = false;
+
+    /**
+     * To tell this converter that this SqlNode tree contains join hint and then a query block alias
+     * will be attached to the root node of the query block.
+     *
+     * <p>The `containsJoinHint` is false default to be compatible with previous behavior and then
+     * planner can reuse some node.
+     *
+     * <p>TODO At present, it is a relatively hacked way
+     */
+    public void containsJoinHint() {
+        containsJoinHint = true;
+    }
+
+    // ----- FLINK MODIFICATION END -----
+
     /**
      * Converts a FROM clause into a relational expression.
      *
@@ -2061,6 +2090,36 @@ public class SqlToRelConverter {
                     }
                 }
                 convertFrom(bb, firstOperand, fieldNameList);
+
+                // ----- FLINK MODIFICATION BEGIN -----
+
+                // Add a query-block alias hint to distinguish different query levels
+                // Due to Calcite will expand the whole SQL Rel Node tree that contains query block,
+                // but sometimes the query block should be perceived such as join hint propagation.
+                // TODO add query-block alias hint in SqlNode instead of here
+                if (containsJoinHint) {
+                    RelNode root = bb.root;
+
+                    if (root instanceof Hintable) {
+                        RelHint queryBlockAliasHint =
+                                RelHint.builder(FlinkHints.HINT_ALIAS)
+                                        .hintOption(call.operand(1).toString())
+                                        .build();
+                        RelNode newRoot =
+                                ((Hintable) root)
+                                        .attachHints(
+                                                Collections.singletonList(queryBlockAliasHint));
+                        boolean isLeaf = leaves.containsKey(root);
+                        if (isLeaf) {
+                            // remove old root node
+                            leaves.remove(root);
+                        }
+
+                        bb.setRoot(newRoot, isLeaf);
+                    }
+                }
+
+                // ----- FLINK MODIFICATION END -----
                 return;
 
             case MATCH_RECOGNIZE:
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttle.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttle.java
new file mode 100644
index 00000000000..426ea68fee2
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttle.java
@@ -0,0 +1,179 @@
+/*
+ * 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.alias;
+
+import org.apache.flink.table.planner.hint.FlinkHints;
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.rel.logical.LogicalJoin;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Deque;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * Due to Calcite will expand the whole SQL RelNode tree that contains query block, join hints will
+ * be propagated from root to leaves in the whole RelNode tree. This shuttle is used to clear the
+ * join hints that are propagated into the query block incorrectly.
+ *
+ * <p>See more at {@see org.apache.calcite.sql2rel.SqlToRelConverter#convertFrom()}.
+ *
+ * <p>TODO some node will be attached join hints when parse SqlNode to RelNode such as Project and
+ * etc. The join hints on these node can also be cleared.
+ */
+public class ClearJoinHintWithInvalidPropagationShuttle extends RelShuttleImpl {
+
+    @Override
+    public RelNode visit(LogicalJoin join) {
+        List<RelHint> hints = join.getHints();
+
+        Set<String> allHintNames =
+                hints.stream().map(hint -> hint.hintName).collect(Collectors.toSet());
+
+        // there are no join hints on this Join node
+        if (allHintNames.stream().noneMatch(JoinStrategy::isJoinStrategy)) {
+            return super.visit(join);
+        }
+
+        Optional<RelHint> firstAliasHint =
+                hints.stream()
+                        .filter(hint -> FlinkHints.HINT_ALIAS.equals(hint.hintName))
+                        .findFirst();
+
+        // there are no alias hints on this Join node
+        if (!firstAliasHint.isPresent()) {
+            return super.visit(join);
+        }
+
+        List<RelHint> joinHintsFromOuterQueryBlock =
+                hints.stream()
+                        .filter(
+                                hint ->
+                                        JoinStrategy.isJoinStrategy(hint.hintName)
+                                                // if the size of inheritPath is bigger than 0, it
+                                                // means that this join hint is propagated from its
+                                                // parent
+                                                && hint.inheritPath.size()
+                                                        > firstAliasHint.get().inheritPath.size())
+                        .collect(Collectors.toList());
+
+        if (joinHintsFromOuterQueryBlock.isEmpty()) {
+            return super.visit(join);
+        }
+
+        RelNode newJoin = join;
+        ClearOuterJoinHintShuttle clearOuterJoinHintShuttle;
+
+        for (RelHint outerJoinHint : joinHintsFromOuterQueryBlock) {
+            clearOuterJoinHintShuttle = new ClearOuterJoinHintShuttle(outerJoinHint);
+            newJoin = newJoin.accept(clearOuterJoinHintShuttle);
+        }
+
+        return super.visit(newJoin);
+    }
+
+    /**
+     * A shuttle to clean the join hints which are in outer query block and should not affect the
+     * query-block inside.
+     */
+    private static class ClearOuterJoinHintShuttle extends RelShuttleImpl {
+        // the current inheritPath about the join hint that need be removed
+        private final Deque<Integer> currentInheritPath;
+
+        // the join hint that need be removed
+        private final RelHint joinHintNeedRemove;
+
+        public ClearOuterJoinHintShuttle(RelHint joinHintNeedRemove) {
+            this.joinHintNeedRemove = joinHintNeedRemove;
+            this.currentInheritPath = new ArrayDeque<>();
+            this.currentInheritPath.addAll(joinHintNeedRemove.inheritPath);
+        }
+
+        @Override
+        protected RelNode visitChild(RelNode parent, int i, RelNode child) {
+            currentInheritPath.addLast(i);
+            RelNode newNode = super.visitChild(parent, i, child);
+            currentInheritPath.removeLast();
+            return newNode;
+        }
+
+        @Override
+        public RelNode visit(LogicalJoin join) {
+            List<RelHint> hints = new ArrayList<>(join.getHints());
+            Optional<RelHint> invalidJoinHint = getInvalidJoinHint(hints);
+
+            // if this join node contains the join hint that needs to be removed
+            if (invalidJoinHint.isPresent()) {
+                hints.remove(invalidJoinHint.get());
+                return super.visit(join.withHints(hints));
+            }
+
+            return super.visit(join);
+        }
+
+        /**
+         * Get the invalid join hint in this node.
+         *
+         * <p>The invalid join meets the following requirement:
+         *
+         * <p>1. This hint name is same with the join hint that needs to be removed
+         *
+         * <p>2.The length of this hint should be same with the length of propagating this removed
+         * join hint.
+         *
+         * <p>3. The inherited path of this hint should match the inherited path of this removed
+         * join hint.
+         *
+         * @param hints all hints
+         * @return return the invalid join hint if exists, else return empty
+         */
+        private Optional<RelHint> getInvalidJoinHint(List<RelHint> hints) {
+            for (RelHint hint : hints) {
+                if (hint.hintName.equals(joinHintNeedRemove.hintName)
+                        && isMatchInvalidInheritPath(
+                                new ArrayList<>(currentInheritPath), hint.inheritPath)) {
+                    return Optional.of(hint);
+                }
+            }
+            return Optional.empty();
+        }
+
+        private boolean isMatchInvalidInheritPath(
+                List<Integer> invalidInheritPath, List<Integer> checkedInheritPath) {
+            if (invalidInheritPath.size() != checkedInheritPath.size()) {
+                return false;
+            }
+
+            for (int i = 0; i < invalidInheritPath.size(); i++) {
+                if (!Objects.equals(invalidInheritPath.get(i), checkedInheritPath.get(i))) {
+                    return false;
+                }
+            }
+            return true;
+        }
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java
index d5cbb555be3..5da3c39c4b8 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java
@@ -307,7 +307,8 @@ public final class RelTimeIndicatorConverter extends RelHomogeneousShuttle {
                                             }
                                         }
                                     });
-            return FlinkLogicalJoin.create(newLeft, newRight, newCondition, join.getJoinType());
+            return FlinkLogicalJoin.create(
+                    newLeft, newRight, newCondition, join.getHints(), join.getJoinType());
         }
     }
 
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java
index 6d8e438d47e..c3fea355336 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java
@@ -20,11 +20,14 @@ package org.apache.flink.table.planner.hint;
 
 import org.apache.flink.table.planner.plan.rules.logical.WrapJsonAggFunctionArgumentsRule;
 
+import org.apache.calcite.rel.hint.HintOptionChecker;
 import org.apache.calcite.rel.hint.HintPredicates;
 import org.apache.calcite.rel.hint.HintStrategy;
 import org.apache.calcite.rel.hint.HintStrategyTable;
 import org.apache.calcite.util.Litmus;
 
+import java.util.Collections;
+
 /** A collection of Flink style {@link HintStrategy}s. */
 public abstract class FlinkHintStrategies {
 
@@ -51,6 +54,57 @@ public abstract class FlinkHintStrategies {
                         HintStrategy.builder(HintPredicates.AGGREGATE)
                                 .excludedRules(WrapJsonAggFunctionArgumentsRule.INSTANCE)
                                 .build())
+                // internal join hint used for alias
+                .hintStrategy(
+                        FlinkHints.HINT_ALIAS,
+                        // currently, only join hints care about query block alias
+                        HintStrategy.builder(HintPredicates.JOIN)
+                                .optionChecker(fixedSizeListOptionChecker(1))
+                                .build())
+                // TODO semi/anti join with CORRELATE is not supported
+                .hintStrategy(
+                        JoinStrategy.BROADCAST.getJoinHintName(),
+                        HintStrategy.builder(HintPredicates.JOIN)
+                                .optionChecker(NON_EMPTY_LIST_OPTION_CHECKER)
+                                .build())
+                .hintStrategy(
+                        JoinStrategy.SHUFFLE_HASH.getJoinHintName(),
+                        HintStrategy.builder(HintPredicates.JOIN)
+                                .optionChecker(NON_EMPTY_LIST_OPTION_CHECKER)
+                                .build())
+                .hintStrategy(
+                        JoinStrategy.SHUFFLE_MERGE.getJoinHintName(),
+                        HintStrategy.builder(HintPredicates.JOIN)
+                                .optionChecker(NON_EMPTY_LIST_OPTION_CHECKER)
+                                .build())
+                .hintStrategy(
+                        JoinStrategy.NEST_LOOP.getJoinHintName(),
+                        HintStrategy.builder(HintPredicates.JOIN)
+                                .optionChecker(NON_EMPTY_LIST_OPTION_CHECKER)
+                                .build())
                 .build();
     }
+
+    private static HintOptionChecker fixedSizeListOptionChecker(int size) {
+        return (hint, errorHandler) ->
+                errorHandler.check(
+                        hint.listOptions.size() == size,
+                        "Invalid hint: {}, expecting {} table or view {} "
+                                + "specified in hint {}.",
+                        FlinkHints.stringifyHints(Collections.singletonList(hint)),
+                        size,
+                        size > 1 ? "names" : "name",
+                        hint.hintName);
+    }
+
+    // ~ hint option checker ------------------------------------------------------------
+
+    private static final HintOptionChecker NON_EMPTY_LIST_OPTION_CHECKER =
+            (hint, errorHandler) ->
+                    errorHandler.check(
+                            hint.listOptions.size() > 0,
+                            "Invalid hint: {}, expecting at least "
+                                    + "one table or view specified in hint {}.",
+                            FlinkHints.stringifyHints(Collections.singletonList(hint)),
+                            hint.hintName);
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java
index 8117e4ae98a..20f0ee2a037 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java
@@ -18,14 +18,24 @@
 
 package org.apache.flink.table.planner.hint;
 
+import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.planner.plan.rules.logical.WrapJsonAggFunctionArgumentsRule;
+import org.apache.flink.table.planner.plan.schema.FlinkPreparingTableBase;
 
+import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.hint.Hintable;
 import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.commons.lang3.StringUtils;
 
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
+import java.util.stream.Collectors;
 
 /** Utility class for Flink hints. */
 public abstract class FlinkHints {
@@ -33,6 +43,9 @@ public abstract class FlinkHints {
 
     public static final String HINT_NAME_OPTIONS = "OPTIONS";
 
+    // ~ Internal alias tag hint
+    public static final String HINT_ALIAS = "ALIAS";
+
     /**
      * Internal hint that JSON aggregation function arguments have been wrapped already. See {@link
      * WrapJsonAggFunctionArgumentsRule}.
@@ -75,4 +88,92 @@ public abstract class FlinkHints {
         newProps.putAll(hints);
         return Collections.unmodifiableMap(newProps);
     }
+
+    public static Optional<String> getTableAlias(RelNode node) {
+        if (node instanceof Hintable) {
+            Hintable aliasNode = (Hintable) node;
+            List<String> aliasNames =
+                    aliasNode.getHints().stream()
+                            .filter(h -> h.hintName.equalsIgnoreCase(FlinkHints.HINT_ALIAS))
+                            .flatMap(h -> h.listOptions.stream())
+                            .collect(Collectors.toList());
+            if (aliasNames.size() > 0) {
+                return Optional.of(aliasNames.get(0));
+            } else if (canTransposeToTableScan(node)) {
+                return getTableAlias(node.getInput(0));
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static boolean canTransposeToTableScan(RelNode node) {
+        // TODO support look up join
+        return node instanceof LogicalProject // computed column on table
+                || node instanceof LogicalFilter;
+    }
+
+    /** Returns the qualified name of a table scan, otherwise returns empty. */
+    public static Optional<String> getTableName(RelOptTable table) {
+        if (table == null) {
+            return Optional.empty();
+        }
+
+        String tableName;
+        if (table instanceof FlinkPreparingTableBase) {
+            tableName = StringUtils.join(((FlinkPreparingTableBase) table).getNames(), '.');
+        } else {
+            throw new TableException(
+                    String.format(
+                            "Could not get the table name with the unknown table class `%s`",
+                            table.getClass().getCanonicalName()));
+        }
+
+        return Optional.of(tableName);
+    }
+
+    public static String stringifyHints(List<RelHint> hints) {
+        StringBuilder sb = new StringBuilder();
+        boolean first = true;
+        for (RelHint h : hints) {
+            if (h.hintName.equalsIgnoreCase(FlinkHints.HINT_ALIAS)) {
+                continue;
+            }
+            if (!first) {
+                sb.append(", ");
+            }
+            sb.append(h.hintName);
+            if (h.listOptions.size() > 0) {
+                String listStr = h.listOptions.stream().collect(Collectors.joining(",", "(", ")"));
+                sb.append(listStr);
+            } else if (h.kvOptions.size() > 0) {
+                String mapStr =
+                        h.kvOptions.entrySet().stream()
+                                .map(e -> e.getKey() + "=" + e.getValue())
+                                .collect(Collectors.joining(", ", "(", ")"));
+                sb.append(mapStr);
+            }
+            first = false;
+        }
+        return sb.toString();
+    }
+
+    /** Get all join hints. */
+    public static List<RelHint> getAllJoinHints(List<RelHint> allHints) {
+        return allHints.stream()
+                .filter(hint -> JoinStrategy.isJoinStrategy(hint.hintName))
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Get all query block alias hints.
+     *
+     * <p>Because query block alias hints will be propagated from root to leaves, so maybe one node
+     * will contain multi alias hints. But only the first one is the really query block name where
+     * this node is.
+     */
+    public static List<RelHint> getQueryBlockAliasHints(List<RelHint> allHints) {
+        return allHints.stream()
+                .filter(hint -> hint.hintName.equals(FlinkHints.HINT_ALIAS))
+                .collect(Collectors.toList());
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/JoinStrategy.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/JoinStrategy.java
new file mode 100644
index 00000000000..cf6345167e0
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/JoinStrategy.java
@@ -0,0 +1,87 @@
+/*
+ * 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.hint;
+
+import java.util.List;
+
+/** Currently available join strategies and corresponding join hint names. */
+public enum JoinStrategy {
+    /**
+     * Instructs the optimizer to use broadcast hash join strategy. If both sides are specified in
+     * this hint, the side that is first written will be broadcast.
+     */
+    BROADCAST("BROADCAST"),
+
+    /**
+     * Instructs the optimizer to use shuffle hash join strategy. If both sides are specified in
+     * this hint, the side that is first written will be treated as the build side.
+     */
+    SHUFFLE_HASH("SHUFFLE_HASH"),
+
+    /**
+     * Instructs the optimizer to use shuffle sort merge join strategy. As long as one of the side
+     * is specified in this hint, it will be tried.
+     */
+    SHUFFLE_MERGE("SHUFFLE_MERGE"),
+
+    /**
+     * Instructs the optimizer to use nest loop join strategy. If both sides are specified in this
+     * hint, the side that is first written will be treated as the build side.
+     */
+    NEST_LOOP("NEST_LOOP");
+
+    private final String joinHintName;
+
+    JoinStrategy(String joinHintName) {
+        this.joinHintName = joinHintName;
+    }
+
+    // ~ option name for join hint
+    public static final String LEFT_INPUT = "LEFT";
+    public static final String RIGHT_INPUT = "RIGHT";
+
+    public static boolean isJoinStrategy(String hintName) {
+        try {
+            JoinStrategy.valueOf(hintName);
+            return true;
+        } catch (Exception e) {
+            return false;
+        }
+    }
+
+    public String getJoinHintName() {
+        return joinHintName;
+    }
+
+    public static boolean validOptions(String hintName, List<String> options) {
+        if (!isJoinStrategy(hintName)) {
+            return false;
+        }
+
+        JoinStrategy strategy = JoinStrategy.valueOf(hintName);
+        switch (strategy) {
+            case SHUFFLE_HASH:
+            case SHUFFLE_MERGE:
+            case BROADCAST:
+            case NEST_LOOP:
+                return options.size() > 0;
+        }
+        return false;
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolver.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolver.java
new file mode 100644
index 00000000000..177c3e34090
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolver.java
@@ -0,0 +1,71 @@
+/*
+ * 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.optimize;
+
+import org.apache.flink.table.planner.hint.FlinkHints;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** A shuttle to remove query block alias hint. */
+public class ClearQueryBlockAliasResolver extends RelShuttleImpl {
+
+    public List<RelNode> resolve(List<RelNode> roots) {
+        return roots.stream().map(node -> node.accept(this)).collect(Collectors.toList());
+    }
+
+    @Override
+    protected RelNode visitChild(RelNode parent, int i, RelNode child) {
+        RelNode newParent = clearQueryBlockAlias(parent);
+        return super.visitChild(newParent, i, child);
+    }
+
+    @Override
+    public RelNode visit(TableScan scan) {
+        RelNode newScan = clearQueryBlockAlias(scan);
+        return super.visit(newScan);
+    }
+
+    private RelNode clearQueryBlockAlias(RelNode relNode) {
+        if (!(relNode instanceof Hintable)) {
+            return relNode;
+        }
+
+        List<RelHint> hints = ((Hintable) relNode).getHints();
+        List<RelHint> newHints = new ArrayList<>();
+        for (RelHint hint : hints) {
+            if (!FlinkHints.HINT_ALIAS.equals(hint.hintName)) {
+                newHints.add(hint);
+            }
+        }
+
+        if (newHints.size() != hints.size()) {
+            return ((Hintable) relNode).withHints(newHints);
+        }
+
+        return relNode;
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java
new file mode 100644
index 00000000000..407d2ce79ab
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java
@@ -0,0 +1,237 @@
+/*
+ * 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.optimize;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.hint.FlinkHints;
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static java.util.Collections.singletonList;
+
+/**
+ * Resolve and validate the join hints.
+ *
+ * <p>Note: duplicate join hints are not checked here.
+ */
+public class JoinHintResolver extends RelShuttleImpl {
+    private final Set<RelHint> allHints = new HashSet<>();
+    private final Set<RelHint> validHints = new HashSet<>();
+
+    /**
+     * Resolves and validates join hints in the given {@link RelNode} list, an {@link
+     * ValidationException} will be raised for invalid hints.
+     *
+     * <p>After resolving join hints, the options of the join hints (declared table name or query
+     * block name) will be replaced to {@link JoinStrategy#LEFT_INPUT} or {@link
+     * JoinStrategy#RIGHT_INPUT}
+     *
+     * <p>If the declared table name or query name in a join hint could not match the left side or
+     * right side of this join, that means this join hint is invalid and a {@link
+     * ValidationException} will be thrown.
+     */
+    public List<RelNode> resolve(List<RelNode> roots) {
+        List<RelNode> resolvedRoots =
+                roots.stream().map(node -> node.accept(this)).collect(Collectors.toList());
+        validateHints();
+        return resolvedRoots;
+    }
+
+    @Override
+    public RelNode visit(LogicalJoin join) {
+        return visitBiRel(join);
+    }
+
+    private RelNode visitBiRel(BiRel biRel) {
+        Optional<String> leftName = extractAliasOrTableName(biRel.getLeft());
+        Optional<String> rightName = extractAliasOrTableName(biRel.getRight());
+
+        Set<RelHint> existentKVHints = new HashSet<>();
+
+        List<RelHint> oldHints = ((Hintable) biRel).getHints();
+        List<RelHint> newHints = new ArrayList<>();
+
+        for (RelHint hint : oldHints) {
+            if (JoinStrategy.isJoinStrategy(hint.hintName)) {
+                allHints.add(trimInheritPath(hint));
+                // the declared table name or query block name is replaced by
+                // JoinStrategy#LEFT_INPUT or JoinStrategy#RIGHT_INPUT
+                List<String> newOptions =
+                        getNewJoinHintOptions(leftName, rightName, hint.listOptions, hint.hintName);
+
+                // check whether the join hints options are valid
+                boolean isValidOption = JoinStrategy.validOptions(hint.hintName, newOptions);
+                if (isValidOption) {
+                    validHints.add(trimInheritPath(hint));
+                    // if the hint defines more than one args, only
+                    // retain the first one
+                    newHints.add(
+                            RelHint.builder(hint.hintName)
+                                    .hintOptions(singletonList(newOptions.get(0)))
+                                    .build());
+                }
+            } else {
+                if (!existentKVHints.contains(hint)) {
+                    existentKVHints.add(hint);
+                    newHints.add(hint);
+                }
+            }
+        }
+
+        RelNode newNode = super.visitChildren(biRel);
+
+        List<RelHint> oldJoinHints = FlinkHints.getAllJoinHints(oldHints);
+        if (!oldJoinHints.isEmpty()) {
+            // replace the table name as LEFT or RIGHT
+            return ((Hintable) newNode).withHints(newHints);
+        }
+        // has no hints, return original node directly.
+        return newNode;
+    }
+
+    private List<String> getNewJoinHintOptions(
+            Optional<String> leftName,
+            Optional<String> rightName,
+            List<String> listOptions,
+            String hintName) {
+        return listOptions.stream()
+                .map(
+                        option -> {
+                            if (leftName.isPresent()
+                                    && rightName.isPresent()
+                                    && matchIdentifier(option, leftName.get())
+                                    && matchIdentifier(option, rightName.get())) {
+                                throw new ValidationException(
+                                        String.format(
+                                                "Ambitious option: %s in hint: %s, the input "
+                                                        + "relations are: %s, %s",
+                                                option, hintName, leftName, rightName));
+                            } else if (leftName.isPresent()
+                                    && matchIdentifier(option, leftName.get())) {
+                                return JoinStrategy.LEFT_INPUT;
+                            } else if (rightName.isPresent()
+                                    && matchIdentifier(option, rightName.get())) {
+                                return JoinStrategy.RIGHT_INPUT;
+                            } else {
+                                return "";
+                            }
+                        })
+                .filter(StringUtils::isNotEmpty)
+                .collect(Collectors.toList());
+    }
+
+    private void validateHints() {
+        Set<RelHint> invalidHints = new HashSet<>(allHints);
+        invalidHints.removeAll(validHints);
+        if (!invalidHints.isEmpty()) {
+            String errorMsg =
+                    invalidHints.stream()
+                            .map(
+                                    hint ->
+                                            hint.hintName
+                                                    + "("
+                                                    + StringUtils.join(hint.listOptions, ", ")
+                                                    + ")`")
+                            .collect(Collectors.joining("\n`", "\n`", ""));
+            throw new ValidationException(
+                    String.format(
+                            "The options of following hints cannot match the name of "
+                                    + "input tables or views: %s",
+                            errorMsg));
+        }
+    }
+
+    private RelHint trimInheritPath(RelHint hint) {
+        RelHint.Builder builder = RelHint.builder(hint.hintName);
+        if (hint.listOptions.isEmpty()) {
+            return builder.hintOptions(hint.kvOptions).build();
+        } else {
+            return builder.hintOptions(hint.listOptions).build();
+        }
+    }
+
+    private Optional<String> extractAliasOrTableName(RelNode node) {
+        // check whether the input relation is converted from a view
+        Optional<String> aliasName = FlinkHints.getTableAlias(node);
+        if (aliasName.isPresent()) {
+            return aliasName;
+        }
+        // otherwise, the option may be a table name
+        Optional<TableScan> tableScan = getTableScan(node);
+        if (tableScan.isPresent()) {
+            Optional<String> tableName = FlinkHints.getTableName(tableScan.get().getTable());
+            if (tableName.isPresent()) {
+                return tableName;
+            }
+        }
+
+        return Optional.empty();
+    }
+
+    private Optional<TableScan> getTableScan(RelNode node) {
+        if (node instanceof TableScan) {
+            return Optional.of((TableScan) node);
+        } else {
+            if (FlinkHints.canTransposeToTableScan(node)) {
+                return getTableScan(node.getInput(0));
+            } else {
+                return Optional.empty();
+            }
+        }
+    }
+
+    /**
+     * Check whether the given hint option matches the table qualified names. For convenience, we
+     * follow a simple rule: the matching is successful if the option is the suffix of the table
+     * qualified names.
+     */
+    private boolean matchIdentifier(String option, String tableIdentifier) {
+        String[] optionNames = option.split("\\.");
+        int optionNameLength = optionNames.length;
+
+        String[] tableNames = tableIdentifier.split("\\.");
+        int tableNameLength = tableNames.length;
+
+        for (int i = 0; i < Math.min(optionNameLength, tableNameLength); i++) {
+            String currOptionName = optionNames[optionNameLength - 1 - i];
+            String currTableName = tableNames[tableNameLength - 1 - i];
+
+            if (!currOptionName.equals(currTableName)) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java
index 225d270f382..2caa2bb951b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java
@@ -394,7 +394,8 @@ public class StreamNonDeterministicUpdatePlanVisitor {
                                                 false,
                                                 true,
                                                 false,
-                                                true));
+                                                true,
+                                                false));
                         throw new TableException(errorMsg.toString());
                     }
                 }
@@ -856,7 +857,8 @@ public class StreamNonDeterministicUpdatePlanVisitor {
                                 false,
                                 true,
                                 false,
-                                true));
+                                true,
+                                false));
 
         throw new TableException(errorMsg.toString());
     }
@@ -897,7 +899,8 @@ public class StreamNonDeterministicUpdatePlanVisitor {
                                 false,
                                 true,
                                 false,
-                                true));
+                                true,
+                                false));
 
         throw new TableException(errorMsg.toString());
     }
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala
index dde9120d942..41ca922b610 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala
@@ -165,7 +165,7 @@ object FlinkLogicalRelFactories {
         variablesSet: util.Set[CorrelationId],
         joinType: JoinRelType,
         semiJoinDone: Boolean): RelNode = {
-      FlinkLogicalJoin.create(left, right, condition, joinType)
+      FlinkLogicalJoin.create(left, right, condition, hints, joinType)
     }
   }
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala
index 7f8292284a7..f9488750b53 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala
@@ -22,8 +22,10 @@ import org.apache.flink.sql.parser.ddl.{SqlCompilePlan, SqlReset, SqlSet, SqlUse
 import org.apache.flink.sql.parser.dml.{RichSqlInsert, SqlBeginStatementSet, SqlCompileAndExecutePlan, SqlEndStatementSet, SqlExecute, SqlExecutePlan, SqlStatementSet}
 import org.apache.flink.sql.parser.dql._
 import org.apache.flink.table.api.{TableException, ValidationException}
+import org.apache.flink.table.planner.hint.JoinStrategy
 import org.apache.flink.table.planner.parse.CalciteParser
 import org.apache.flink.table.planner.plan.FlinkCalciteCatalogReader
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil
 
 import com.google.common.collect.ImmutableList
 import org.apache.calcite.config.NullCollation
@@ -33,8 +35,9 @@ import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.{RelFieldCollation, RelRoot}
 import org.apache.calcite.rel.hint.RelHint
 import org.apache.calcite.rex.{RexInputRef, RexNode}
-import org.apache.calcite.sql.{SqlInsert, SqlKind, SqlNode, SqlOperatorTable}
+import org.apache.calcite.sql.{SqlCall, SqlHint, SqlKind, SqlNode, SqlNodeList, SqlOperatorTable, SqlSelect, SqlTableRef}
 import org.apache.calcite.sql.advise.SqlAdvisorValidator
+import org.apache.calcite.sql.util.SqlShuttle
 import org.apache.calcite.sql.validate.SqlValidator
 import org.apache.calcite.sql2rel.{SqlRexConvertletTable, SqlToRelConverter}
 import org.apache.calcite.tools.{FrameworkConfig, RelConversionException}
@@ -192,6 +195,17 @@ class FlinkPlannerImpl(
       assert(validatedSqlNode != null)
       val sqlToRelConverter: SqlToRelConverter = createSqlToRelConverter(sqlValidator)
 
+      // check whether this SqlNode tree contains join hints
+      val checkContainJoinHintShuttle = new CheckContainJoinHintShuttle
+      validatedSqlNode.accept(checkContainJoinHintShuttle)
+      checkContainJoinHintShuttle.containsJoinHint
+
+      // TODO currently, it is a relatively hacked way to tell converter
+      // that this SqlNode tree contains join hints
+      if (checkContainJoinHintShuttle.containsJoinHint) {
+        sqlToRelConverter.containsJoinHint()
+      }
+
       sqlToRelConverter.convertQuery(validatedSqlNode, false, true)
       // we disable automatic flattening in order to let composite types pass without modification
       // we might enable it again once Calcite has better support for structured types
@@ -206,6 +220,39 @@ class FlinkPlannerImpl(
     }
   }
 
+  class CheckContainJoinHintShuttle extends SqlShuttle {
+    var containsJoinHint: Boolean = false
+
+    override def visit(call: SqlCall): SqlNode = {
+      call match {
+        case select: SqlSelect =>
+          if (select.hasHints && hasJoinHint(select.getHints.getList)) {
+            containsJoinHint = true
+            return call
+          }
+        case table: SqlTableRef =>
+          val hintList = table.getOperandList.get(1).asInstanceOf[SqlNodeList]
+          if (hasJoinHint(hintList.getList)) {
+            containsJoinHint = true
+            return call
+          }
+        case _ => // ignore
+      }
+      super.visit(call)
+    }
+
+    private def hasJoinHint(hints: util.List[SqlNode]): Boolean = {
+      JavaScalaConversionUtil.toScala(hints).foreach {
+        case hint: SqlHint =>
+          val hintName = hint.getName
+          if (JoinStrategy.isJoinStrategy(hintName)) {
+            return true
+          }
+      }
+      false
+    }
+  }
+
   def validateExpression(
       sqlNode: SqlNode,
       inputRowType: RelDataType,
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Sink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Sink.scala
index c4d9d0bef10..13a76581afc 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Sink.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Sink.scala
@@ -17,7 +17,7 @@
  */
 package org.apache.flink.table.planner.plan.nodes.calcite
 
-import org.apache.flink.table.catalog.{ContextResolvedTable, ObjectIdentifier, ResolvedCatalogTable}
+import org.apache.flink.table.catalog.ContextResolvedTable
 import org.apache.flink.table.connector.sink.DynamicTableSink
 import org.apache.flink.table.planner.plan.utils.RelExplainUtil
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala
index 0aff68e8980..8f1d42bd3c7 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala
@@ -17,6 +17,7 @@
  */
 package org.apache.flink.table.planner.plan.nodes.logical
 
+import org.apache.flink.table.planner.JList
 import org.apache.flink.table.planner.plan.nodes.FlinkConventions
 
 import org.apache.calcite.plan._
@@ -28,8 +29,6 @@ import org.apache.calcite.rel.logical.LogicalJoin
 import org.apache.calcite.rel.metadata.RelMetadataQuery
 import org.apache.calcite.rex.RexNode
 
-import java.util.Collections
-
 import scala.collection.JavaConversions._
 
 /**
@@ -42,16 +41,9 @@ class FlinkLogicalJoin(
     left: RelNode,
     right: RelNode,
     condition: RexNode,
+    hints: JList[RelHint],
     joinType: JoinRelType)
-  extends Join(
-    cluster,
-    traitSet,
-    Collections.emptyList[RelHint](),
-    left,
-    right,
-    condition,
-    Set.empty[CorrelationId],
-    joinType)
+  extends Join(cluster, traitSet, hints, left, right, condition, Set.empty[CorrelationId], joinType)
   with FlinkLogicalRel {
 
   override def copy(
@@ -61,7 +53,7 @@ class FlinkLogicalJoin(
       right: RelNode,
       joinType: JoinRelType,
       semiJoinDone: Boolean): Join = {
-    new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, joinType)
+    new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, getHints, joinType)
   }
 
   override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = {
@@ -97,7 +89,7 @@ private class FlinkLogicalJoinConverter
     val join = rel.asInstanceOf[LogicalJoin]
     val newLeft = RelOptRule.convert(join.getLeft, FlinkConventions.LOGICAL)
     val newRight = RelOptRule.convert(join.getRight, FlinkConventions.LOGICAL)
-    FlinkLogicalJoin.create(newLeft, newRight, join.getCondition, join.getJoinType)
+    FlinkLogicalJoin.create(newLeft, newRight, join.getCondition, join.getHints, join.getJoinType)
   }
 }
 
@@ -108,9 +100,10 @@ object FlinkLogicalJoin {
       left: RelNode,
       right: RelNode,
       conditionExpr: RexNode,
+      hints: JList[RelHint],
       joinType: JoinRelType): FlinkLogicalJoin = {
     val cluster = left.getCluster
     val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL).simplify()
-    new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, joinType)
+    new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, hints, joinType)
   }
 }
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
index 107110b36fe..a4d0baa1912 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
@@ -25,6 +25,7 @@ import org.apache.flink.table.planner.delegation.BatchPlanner
 import org.apache.flink.table.planner.plan.nodes.calcite.{LegacySink, Sink}
 import org.apache.flink.table.planner.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram}
 import org.apache.flink.table.planner.plan.schema.IntermediateRelTable
+import org.apache.flink.table.planner.utils.JavaScalaConversionUtil.{toJava, toScala}
 import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapContext
 import org.apache.flink.table.planner.utils.TableConfigUtils
 import org.apache.flink.util.Preconditions
@@ -38,8 +39,20 @@ class BatchCommonSubGraphBasedOptimizer(planner: BatchPlanner)
   extends CommonSubGraphBasedOptimizer {
 
   override protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock] = {
+    // TODO currently join hint only works in BATCH
+    // resolve hints before optimizing
+    val joinHintResolver = new JoinHintResolver()
+    val resolvedHintRoots = joinHintResolver.resolve(toJava(roots))
+
+    // clear query block alias before optimizing
+    val clearQueryBlockAliasResolver = new ClearQueryBlockAliasResolver
+    val resolvedAliasRoots = clearQueryBlockAliasResolver.resolve(resolvedHintRoots)
+
     // build RelNodeBlock plan
-    val rootBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, planner.getTableConfig)
+    val rootBlocks =
+      RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(
+        toScala(resolvedAliasRoots),
+        planner.getTableConfig)
     // optimize recursively RelNodeBlock
     rootBlocks.foreach(optimizeBlock)
     rootBlocks
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala
index 43f06975492..376717c84a6 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala
@@ -60,6 +60,7 @@ class SplitPythonConditionFromJoinRule
       join.getLeft,
       join.getRight,
       newJoinCondition,
+      join.getHints,
       join.getJoinType)
 
     val rexProgram = new RexProgramBuilder(bottomJoin.getRowType, rexBuilder).getProgram
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRule.scala
index 21866df620d..d3ef36ca150 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRule.scala
@@ -110,7 +110,12 @@ class TemporalJoinRewriteWithUniqueKeyRule
       }
     })
     val rewriteJoin =
-      FlinkLogicalJoin.create(leftInput, snapshot, newJoinCondition, join.getJoinType)
+      FlinkLogicalJoin.create(
+        leftInput,
+        snapshot,
+        newJoinCondition,
+        join.getHints,
+        join.getJoinType)
     call.transformTo(rewriteJoin)
   }
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala
index 2eb6fb6ce3b..4e22525e449 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala
@@ -17,16 +17,13 @@
  */
 package org.apache.flink.table.planner.plan.rules.physical.batch
 
-import org.apache.flink.configuration.ReadableConfig
-import org.apache.flink.table.api.config.OptimizerConfigOptions
-import org.apache.flink.table.planner.JDouble
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.planner.hint.JoinStrategy
 import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.planner.plan.nodes.FlinkConventions
 import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalHashJoin
-import org.apache.flink.table.planner.plan.utils.OperatorType
 import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig
-import org.apache.flink.table.planner.utils.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
 import org.apache.calcite.plan.RelOptRule.{any, operand}
@@ -50,28 +47,15 @@ class BatchPhysicalHashJoinRule
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val join: Join = call.rel(0)
-    val joinInfo = join.analyzeCondition
-    // join keys must not be empty
-    if (joinInfo.pairs().isEmpty) {
-      return false
-    }
-
-    val tableConfig = unwrapTableConfig(call)
-    val isShuffleHashJoinEnabled = !isOperatorDisabled(tableConfig, OperatorType.ShuffleHashJoin)
-    val isBroadcastHashJoinEnabled =
-      !isOperatorDisabled(tableConfig, OperatorType.BroadcastHashJoin)
+    val tableConfig = unwrapTableConfig(join)
 
-    val leftSize = binaryRowRelNodeSize(join.getLeft)
-    val rightSize = binaryRowRelNodeSize(join.getRight)
-    val (isBroadcast, _) = canBroadcast(join.getJoinType, leftSize, rightSize, tableConfig)
-
-    // TODO use shuffle hash join if isBroadcast is true and isBroadcastHashJoinEnabled is false ?
-    if (isBroadcast) isBroadcastHashJoinEnabled else isShuffleHashJoinEnabled
+    canUseJoinStrategy(join, tableConfig, JoinStrategy.BROADCAST) ||
+    canUseJoinStrategy(join, tableConfig, JoinStrategy.SHUFFLE_HASH)
   }
 
   override def onMatch(call: RelOptRuleCall): Unit = {
-    val tableConfig = unwrapTableConfig(call)
     val join: Join = call.rel(0)
+    val tableConfig = unwrapTableConfig(join)
     val joinInfo = join.analyzeCondition
     val joinType = join.getJoinType
 
@@ -89,19 +73,40 @@ class BatchPhysicalHashJoinRule
       case _ => (join.getRight, false)
     }
 
-    val leftSize = binaryRowRelNodeSize(left)
-    val rightSize = binaryRowRelNodeSize(right)
-
-    val (isBroadcast, leftIsBroadcast) = canBroadcast(joinType, leftSize, rightSize, tableConfig)
+    val firstValidJoinHintOp = getFirstValidJoinHint(join, tableConfig)
+
+    val (isBroadcast: Boolean, isLeftToBroadcastOrBuild: Boolean) = firstValidJoinHintOp match {
+      case Some(firstValidJoinHint) =>
+        firstValidJoinHint match {
+          case JoinStrategy.BROADCAST =>
+            val (_, isLeftToBroadcast: Boolean) =
+              checkBroadcast(join, tableConfig, withBroadcastHint = true)
+            (true, isLeftToBroadcast)
+          case JoinStrategy.SHUFFLE_HASH =>
+            val (_, isLeftToBuild: Boolean) =
+              checkShuffleHash(join, tableConfig, withShuffleHashHint = true)
+            (false, isLeftToBuild)
+          case _ =>
+            // this should not happen
+            throw new TableException(
+              String.format(
+                "The planner is trying to convert the " +
+                  "`FlinkLogicalJoin` using BROADCAST or SHUFFLE_HASH," +
+                  " but the first valid join hint is not BROADCAST or SHUFFLE_HASH: %s",
+                firstValidJoinHint
+              ))
+        }
+      case None =>
+        // treat as non-join-hints
+        val (canBroadcast, isLeftToBroadcast) =
+          checkBroadcast(join, tableConfig, withBroadcastHint = false)
 
-    val leftIsBuild = if (isBroadcast) {
-      leftIsBroadcast
-    } else if (leftSize == null || rightSize == null || leftSize == rightSize) {
-      // use left to build hash table if leftSize or rightSize is unknown or equal size.
-      // choose right to build if join is SEMI/ANTI.
-      !join.getJoinType.projectsRight
-    } else {
-      leftSize < rightSize
+        if (canBroadcast) {
+          (true, isLeftToBroadcast)
+        } else {
+          val (_, isLeftToBuild) = checkShuffleHash(join, tableConfig, withShuffleHashHint = false)
+          (false, isLeftToBuild)
+        }
     }
 
     def transformToEquiv(leftRequiredTrait: RelTraitSet, rightRequiredTrait: RelTraitSet): Unit = {
@@ -116,7 +121,7 @@ class BatchPhysicalHashJoinRule
         newRight,
         join.getCondition,
         join.getJoinType,
-        leftIsBuild,
+        isLeftToBroadcastOrBuild,
         isBroadcast,
         tryDistinctBuildRow)
 
@@ -128,7 +133,7 @@ class BatchPhysicalHashJoinRule
       val buildTrait = join.getTraitSet
         .replace(FlinkConventions.BATCH_PHYSICAL)
         .replace(FlinkRelDistribution.BROADCAST_DISTRIBUTED)
-      if (leftIsBroadcast) {
+      if (isLeftToBroadcastOrBuild) {
         transformToEquiv(buildTrait, probeTrait)
       } else {
         transformToEquiv(probeTrait, buildTrait)
@@ -157,40 +162,6 @@ class BatchPhysicalHashJoinRule
 
   }
 
-  /**
-   * Decides whether the join can convert to BroadcastHashJoin.
-   *
-   * @param joinType
-   *   flink join type
-   * @param leftSize
-   *   size of join left child
-   * @param rightSize
-   *   size of join right child
-   * @return
-   *   an Tuple2 instance. The first element of tuple is true if join can convert to broadcast hash
-   *   join, false else. The second element of tuple is true if left side used as broadcast side,
-   *   false else.
-   */
-  private def canBroadcast(
-      joinType: JoinRelType,
-      leftSize: JDouble,
-      rightSize: JDouble,
-      tableConfig: ReadableConfig): (Boolean, Boolean) = {
-    // if leftSize or rightSize is unknown, cannot use broadcast
-    if (leftSize == null || rightSize == null) {
-      return (false, false)
-    }
-    val threshold = tableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD)
-    joinType match {
-      case JoinRelType.LEFT => (rightSize <= threshold, false)
-      case JoinRelType.RIGHT => (leftSize <= threshold, true)
-      case JoinRelType.FULL => (false, false)
-      case JoinRelType.INNER =>
-        (leftSize <= threshold || rightSize <= threshold, leftSize < rightSize)
-      // left side cannot be used as build side in SEMI/ANTI join.
-      case JoinRelType.SEMI | JoinRelType.ANTI => (rightSize <= threshold, false)
-    }
-  }
 }
 
 object BatchPhysicalHashJoinRule {
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala
index a0af70041f1..94cf8884a67 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala
@@ -20,19 +20,43 @@ package org.apache.flink.table.planner.plan.rules.physical.batch
 import org.apache.flink.annotation.Experimental
 import org.apache.flink.configuration.ConfigOption
 import org.apache.flink.configuration.ConfigOptions.key
+import org.apache.flink.table.api.{TableConfig, TableException, ValidationException}
+import org.apache.flink.table.api.config.OptimizerConfigOptions
+import org.apache.flink.table.planner.JDouble
+import org.apache.flink.table.planner.hint.JoinStrategy
 import org.apache.flink.table.planner.plan.nodes.FlinkConventions
 import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalLocalHashAggregate
-import org.apache.flink.table.planner.plan.utils.FlinkRelMdUtil
+import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, OperatorType}
 import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig
+import org.apache.flink.table.planner.utils.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.RelOptRule
 import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core.{Join, JoinRelType}
 import org.apache.calcite.util.ImmutableBitSet
 
 import java.lang.{Boolean => JBoolean, Double => JDouble}
 
+import scala.collection.JavaConversions._
+
 trait BatchPhysicalJoinRuleBase {
 
+  protected def canUseJoinStrategy(
+      join: Join,
+      tableConfig: TableConfig,
+      joinStrategy: JoinStrategy): Boolean = {
+    val firstValidJoinHint = getFirstValidJoinHint(join, tableConfig)
+    if (firstValidJoinHint.nonEmpty) {
+      // if there are join hints, the first hint must be this one, otherwise it is invalid
+      firstValidJoinHint.get.equals(joinStrategy)
+    } else {
+      // if there are no join hints, treat as non-join-hints
+      val (isValid, _) =
+        checkJoinStrategyValid(join, tableConfig, joinStrategy, withHint = false)
+      isValid
+    }
+  }
+
   def addLocalDistinctAgg(node: RelNode, distinctKeys: Seq[Int]): RelNode = {
     val localRequiredTraitSet = node.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL)
     val newInput = RelOptRule.convert(node, localRequiredTraitSet)
@@ -72,6 +96,238 @@ trait BatchPhysicalJoinRuleBase {
       rowCount * FlinkRelMdUtil.binaryRowAverageSize(relNode)
     }
   }
+
+  protected def getFirstValidJoinHint(
+      join: Join,
+      tableConfig: TableConfig): Option[JoinStrategy] = {
+    val allHints = join.getHints
+
+    allHints.forEach(
+      relHint => {
+        if (JoinStrategy.isJoinStrategy(relHint.hintName)) {
+          val joinStrategy = JoinStrategy.valueOf(relHint.hintName)
+          val (isValid, _) =
+            checkJoinStrategyValid(join, tableConfig, joinStrategy, withHint = true)
+          if (isValid) {
+            return Some(joinStrategy)
+          }
+        }
+      })
+
+    None
+  }
+
+  /**
+   * Check whether the join strategy is valid.
+   *
+   * @param join
+   *   the join node
+   * @param tableConfig
+   *   the table config
+   * @param triedJoinStrategy
+   *   the join strategy checked
+   * @param withHint
+   *   whether this check is called with hint
+   * @return
+   *   an Tuple2 instance. The first element of tuple is true if join is valid, false else. The
+   *   second element of tuple is true if left side used as build side, false else.
+   */
+  def checkJoinStrategyValid(
+      join: Join,
+      tableConfig: TableConfig,
+      triedJoinStrategy: JoinStrategy,
+      withHint: Boolean): (Boolean, Boolean) = {
+
+    // TODO currently join hint is not supported with semi/anti join
+    if (withHint && !join.getJoinType.projectsRight()) {
+      return (false, false)
+    }
+
+    triedJoinStrategy match {
+      case JoinStrategy.BROADCAST =>
+        checkBroadcast(join, tableConfig, withHint)
+
+      case JoinStrategy.SHUFFLE_HASH =>
+        checkShuffleHash(join, tableConfig, withHint)
+
+      case JoinStrategy.SHUFFLE_MERGE =>
+        // for SortMergeJoin, there is no diff between with hint or without hint
+        // the second arg should be ignored
+        (checkSortMergeJoin(join, tableConfig), false)
+
+      case JoinStrategy.NEST_LOOP =>
+        checkNestLoopJoin(join, tableConfig, withHint)
+
+      case _ =>
+        throw new ValidationException("Unknown join strategy : " + triedJoinStrategy)
+    }
+  }
+
+  private def isEquivJoin(join: Join): Boolean = {
+    val joinInfo = join.analyzeCondition
+    !joinInfo.pairs().isEmpty
+  }
+
+  /**
+   * Decides whether the join can convert to BroadcastHashJoin.
+   *
+   * @param join
+   *   the join node
+   * @return
+   *   an Tuple2 instance. The first element of tuple is true if join can convert to broadcast hash
+   *   join, false else. The second element of tuple is true if left side used as broadcast side,
+   *   false else.
+   */
+  protected def checkBroadcast(
+      join: Join,
+      tableConfig: TableConfig,
+      withBroadcastHint: Boolean): (Boolean, Boolean) = {
+
+    if (!isEquivJoin(join) || isOperatorDisabled(tableConfig, OperatorType.BroadcastHashJoin)) {
+      return (false, false)
+    }
+
+    // if it is with hint, try best to use it and only check the join type
+    if (withBroadcastHint) {
+      // BROADCAST use first arg as the broadcast side
+      val isLeftToBroadcastInHint =
+        getFirstArgInJoinHint(join, JoinStrategy.BROADCAST.getJoinHintName)
+          .equals(JoinStrategy.LEFT_INPUT)
+
+      join.getJoinType match {
+        // if left join, must broadcast right side
+        case JoinRelType.LEFT => (!isLeftToBroadcastInHint, false)
+        // if right join, must broadcast left side
+        case JoinRelType.RIGHT => (isLeftToBroadcastInHint, true)
+        case JoinRelType.FULL => (false, false)
+        case JoinRelType.INNER =>
+          (true, isLeftToBroadcastInHint)
+        case JoinRelType.SEMI | JoinRelType.ANTI =>
+          // TODO currently join hint is not supported with semi/anti join
+          (false, false)
+      }
+    } else {
+      val leftSize = binaryRowRelNodeSize(join.getLeft)
+      val rightSize = binaryRowRelNodeSize(join.getRight)
+
+      // if it is not with hint, just check size of left and right side by statistic and config
+      // if leftSize or rightSize is unknown, cannot use broadcast
+      if (leftSize == null || rightSize == null) {
+        return (false, false)
+      }
+
+      val threshold =
+        tableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD)
+
+      val rightSizeSmallerThanThreshold = rightSize <= threshold
+      val leftSizeSmallerThanThreshold = leftSize <= threshold
+      val leftSmallerThanRight = leftSize < rightSize
+
+      join.getJoinType match {
+        case JoinRelType.LEFT => (rightSizeSmallerThanThreshold, false)
+        case JoinRelType.RIGHT => (leftSizeSmallerThanThreshold, true)
+        case JoinRelType.FULL => (false, false)
+        case JoinRelType.INNER =>
+          (
+            leftSizeSmallerThanThreshold
+              || rightSizeSmallerThanThreshold,
+            leftSmallerThanRight)
+        // left side cannot be used as build side in SEMI/ANTI join.
+        case JoinRelType.SEMI | JoinRelType.ANTI =>
+          (rightSizeSmallerThanThreshold, false)
+      }
+    }
+  }
+
+  protected def checkShuffleHash(
+      join: Join,
+      tableConfig: TableConfig,
+      withShuffleHashHint: Boolean): (Boolean, Boolean) = {
+    if (!isEquivJoin(join) || isOperatorDisabled(tableConfig, OperatorType.ShuffleHashJoin)) {
+      return (false, false)
+    }
+
+    if (withShuffleHashHint) {
+      val isLeftToBuild = getFirstArgInJoinHint(join, JoinStrategy.SHUFFLE_HASH.getJoinHintName)
+        .equals(JoinStrategy.LEFT_INPUT)
+      (true, isLeftToBuild)
+    } else {
+      val leftSize = binaryRowRelNodeSize(join.getLeft)
+      val rightSize = binaryRowRelNodeSize(join.getRight)
+      val leftIsBuild = if (leftSize == null || rightSize == null || leftSize == rightSize) {
+        // use left to build hash table if leftSize or rightSize is unknown or equal size.
+        // choose right to build if join is SEMI/ANTI.
+        !join.getJoinType.projectsRight
+      } else {
+        leftSize < rightSize
+      }
+      (true, leftIsBuild)
+
+    }
+  }
+
+  // the sort merge join doesn't distinct the build side
+  protected def checkSortMergeJoin(join: Join, tableConfig: TableConfig): Boolean = {
+    if (!isEquivJoin(join) || isOperatorDisabled(tableConfig, OperatorType.SortMergeJoin)) {
+      false
+    } else {
+      true
+    }
+  }
+
+  protected def checkNestLoopJoin(
+      join: Join,
+      tableConfig: TableConfig,
+      withNestLoopHint: Boolean): (Boolean, Boolean) = {
+
+    if (isOperatorDisabled(tableConfig, OperatorType.NestedLoopJoin)) {
+      return (false, false)
+    }
+
+    val isLeftToBuild = if (withNestLoopHint) {
+      getFirstArgInJoinHint(join, JoinStrategy.NEST_LOOP.getJoinHintName)
+        .equals(JoinStrategy.LEFT_INPUT)
+    } else {
+      join.getJoinType match {
+        case JoinRelType.LEFT => false
+        case JoinRelType.RIGHT => true
+        case JoinRelType.INNER | JoinRelType.FULL =>
+          val leftSize = binaryRowRelNodeSize(join.getLeft)
+          val rightSize = binaryRowRelNodeSize(join.getRight)
+          // use left as build size if leftSize or rightSize is unknown.
+          if (leftSize == null || rightSize == null) {
+            true
+          } else {
+            leftSize <= rightSize
+          }
+        case JoinRelType.SEMI | JoinRelType.ANTI => false
+      }
+
+    }
+
+    // all join can use NEST LOOP JOIN
+    (true, isLeftToBuild)
+
+  }
+
+  private def getFirstArgInJoinHint(join: Join, joinHintName: String): String = {
+    join.getHints.forEach(
+      hint => {
+        if (hint.hintName.equals(joinHintName)) {
+          return hint.listOptions.get(0)
+        }
+      })
+
+    // can not happen
+    throw new TableException(
+      String.format(
+        "Fail to find the join hint `%s` among `%s`",
+        joinHintName,
+        join.getHints
+          .map(hint => hint.hintName)
+          .mkString(",")
+      ))
+  }
 }
 object BatchPhysicalJoinRuleBase {
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalNestedLoopJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalNestedLoopJoinRule.scala
index ba72542a32f..ca914cf69fd 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalNestedLoopJoinRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalNestedLoopJoinRule.scala
@@ -17,17 +17,19 @@
  */
 package org.apache.flink.table.planner.plan.rules.physical.batch
 
+import org.apache.flink.table.api.TableException
+import org.apache.flink.table.planner.hint.JoinStrategy
 import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalNestedLoopJoin
-import org.apache.flink.table.planner.plan.utils.OperatorType
 import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig
-import org.apache.flink.table.planner.utils.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.rel.core.{Join, JoinRelType}
 
+import scala.collection.JavaConversions._
+
 /**
  * Rule that converts [[FlinkLogicalJoin]] to [[BatchPhysicalNestedLoopJoin]] if NestedLoopJoin is
  * enabled.
@@ -40,12 +42,14 @@ class BatchPhysicalNestedLoopJoinRule
   with BatchPhysicalNestedLoopJoinRuleBase {
 
   override def matches(call: RelOptRuleCall): Boolean = {
-    val tableConfig = unwrapTableConfig(call)
-    !isOperatorDisabled(tableConfig, OperatorType.NestedLoopJoin)
+    val join: Join = call.rel(0)
+    val tableConfig = unwrapTableConfig(join)
+    canUseJoinStrategy(join, tableConfig, JoinStrategy.NEST_LOOP)
   }
 
   override def onMatch(call: RelOptRuleCall): Unit = {
     val join: Join = call.rel(0)
+    val tableConfig = unwrapTableConfig(join)
     val left = join.getLeft
     val right = join.getJoinType match {
       case JoinRelType.SEMI | JoinRelType.ANTI =>
@@ -59,26 +63,33 @@ class BatchPhysicalNestedLoopJoinRule
         }
       case _ => join.getRight
     }
-    val leftIsBuild = isLeftBuild(join, left, right)
-    val newJoin = createNestedLoopJoin(join, left, right, leftIsBuild, singleRowJoin = false)
-    call.transformTo(newJoin)
-  }
 
-  private def isLeftBuild(join: Join, left: RelNode, right: RelNode): Boolean = {
-    join.getJoinType match {
-      case JoinRelType.LEFT => false
-      case JoinRelType.RIGHT => true
-      case JoinRelType.INNER | JoinRelType.FULL =>
-        val leftSize = binaryRowRelNodeSize(left)
-        val rightSize = binaryRowRelNodeSize(right)
-        // use left as build size if leftSize or rightSize is unknown.
-        if (leftSize == null || rightSize == null) {
-          true
-        } else {
-          leftSize <= rightSize
+    val firstValidJoinHintOp = getFirstValidJoinHint(join, tableConfig)
+
+    val temJoin = join.copy(join.getTraitSet, List(left, right))
+
+    val isLeftToBuild = firstValidJoinHintOp match {
+      case Some(firstValidJoinHint) =>
+        firstValidJoinHint match {
+          case JoinStrategy.NEST_LOOP =>
+            val (_, isLeft) = checkNestLoopJoin(temJoin, tableConfig, withNestLoopHint = true)
+            isLeft
+          case _ =>
+            // this should not happen
+            throw new TableException(String.format(
+              "The planner is trying to convert the " +
+                "`FlinkLogicalJoin` using NEST_LOOP, but the valid join hint is not NEST_LOOP: %s",
+              firstValidJoinHint
+            ))
         }
-      case JoinRelType.SEMI | JoinRelType.ANTI => false
+      case None =>
+        // treat as non-join-hints
+        val (_, isLeft) = checkNestLoopJoin(temJoin, tableConfig, withNestLoopHint = false)
+        isLeft
     }
+
+    val newJoin = createNestedLoopJoin(join, left, right, isLeftToBuild, singleRowJoin = false)
+    call.transformTo(newJoin)
   }
 }
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSingleRowJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSingleRowJoinRule.scala
index 01625a8dda8..904802f2034 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSingleRowJoinRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSingleRowJoinRule.scala
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.rules.physical.batch
 import org.apache.flink.table.planner.plan.nodes.FlinkConventions
 import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalNestedLoopJoin
+import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
 import org.apache.calcite.plan.volcano.RelSubset
@@ -42,13 +43,21 @@ class BatchPhysicalSingleRowJoinRule
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val join: Join = call.rel(0)
-    join.getJoinType match {
-      case JoinRelType.INNER | JoinRelType.FULL =>
-        isSingleRow(join.getLeft) || isSingleRow(join.getRight)
-      case JoinRelType.LEFT => isSingleRow(join.getRight)
-      case JoinRelType.RIGHT => isSingleRow(join.getLeft)
-      case JoinRelType.SEMI | JoinRelType.ANTI => isSingleRow(join.getRight)
-      case _ => false
+    val tableConfig = unwrapTableConfig(join)
+    val firstValidJoinHintOp = getFirstValidJoinHint(join, tableConfig)
+
+    firstValidJoinHintOp match {
+      // the valid join hint keeps higher priority
+      case Some(_) => false
+      case None =>
+        join.getJoinType match {
+          case JoinRelType.INNER | JoinRelType.FULL =>
+            isSingleRow(join.getLeft) || isSingleRow(join.getRight)
+          case JoinRelType.LEFT => isSingleRow(join.getRight)
+          case JoinRelType.RIGHT => isSingleRow(join.getLeft)
+          case JoinRelType.SEMI | JoinRelType.ANTI => isSingleRow(join.getRight)
+          case _ => false
+        }
     }
   }
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala
index a43432a3ebb..676a2aac6a0 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala
@@ -20,13 +20,13 @@ package org.apache.flink.table.planner.plan.rules.physical.batch
 import org.apache.flink.annotation.Experimental
 import org.apache.flink.configuration.ConfigOption
 import org.apache.flink.configuration.ConfigOptions.key
+import org.apache.flink.table.planner.hint.JoinStrategy
 import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.planner.plan.nodes.FlinkConventions
 import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalSortMergeJoin
-import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, OperatorType}
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil
 import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig
-import org.apache.flink.table.planner.utils.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
 import org.apache.calcite.plan.RelOptRule.{any, operand}
@@ -50,10 +50,8 @@ class BatchPhysicalSortMergeJoinRule
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val join: Join = call.rel(0)
-    val joinInfo = join.analyzeCondition
-    val tableConfig = unwrapTableConfig(call)
-    val isSortMergeJoinEnabled = !isOperatorDisabled(tableConfig, OperatorType.SortMergeJoin)
-    !joinInfo.pairs().isEmpty && isSortMergeJoinEnabled
+    val tableConfig = unwrapTableConfig(join)
+    canUseJoinStrategy(join, tableConfig, JoinStrategy.SHUFFLE_MERGE)
   }
 
   override def onMatch(call: RelOptRuleCall): Unit = {
@@ -107,7 +105,7 @@ class BatchPhysicalSortMergeJoinRule
       call.transformTo(newJoin)
     }
 
-    val tableConfig = unwrapTableConfig(call)
+    val tableConfig = unwrapTableConfig(join)
     val candidates =
       if (tableConfig.get(BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED)) {
         // add more possibility to remove redundant sort, and longer optimization time
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
index 848c990bf7b..a971a671fe3 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
@@ -70,7 +70,8 @@ object FlinkRelOptUtil {
       withIdPrefix: Boolean = false,
       withChangelogTraits: Boolean = false,
       withRowType: Boolean = false,
-      withUpsertKey: Boolean = false): String = {
+      withUpsertKey: Boolean = false,
+      withQueryBlockAlias: Boolean = false): String = {
     if (rel == null) {
       return null
     }
@@ -82,7 +83,9 @@ object FlinkRelOptUtil {
       withChangelogTraits,
       withRowType,
       withTreeStyle = true,
-      withUpsertKey)
+      withUpsertKey,
+      withJoinHint = true,
+      withQueryBlockAlias)
     rel.explain(planWriter)
     sw.toString
   }
@@ -143,7 +146,8 @@ object FlinkRelOptUtil {
         // expressions have different types
         withRowType = true,
         // ignore tree style, only contains RelNode's attributes
-        withTreeStyle = false))
+        withTreeStyle = false,
+        withJoinHint = true))
     sw.toString
   }
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala
index 1a64427e2ba..9a7328b91c1 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala
@@ -17,11 +17,14 @@
  */
 package org.apache.flink.table.planner.plan.utils
 
+import org.apache.flink.table.planner.hint.FlinkHints
 import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery
 import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel
 
 import org.apache.calcite.rel.RelNode
+import org.apache.calcite.rel.core.{Join, TableScan}
 import org.apache.calcite.rel.externalize.RelWriterImpl
+import org.apache.calcite.rel.hint.Hintable
 import org.apache.calcite.sql.SqlExplainLevel
 import org.apache.calcite.util.Pair
 
@@ -38,7 +41,9 @@ class RelTreeWriterImpl(
     withChangelogTraits: Boolean = false,
     withRowType: Boolean = false,
     withTreeStyle: Boolean = true,
-    withUpsertKey: Boolean = false)
+    withUpsertKey: Boolean = false,
+    withJoinHint: Boolean = true,
+    withQueryBlockAlias: Boolean = false)
   extends RelWriterImpl(pw, explainLevel, withIdPrefix) {
 
   var lastChildren: Seq[Boolean] = Nil
@@ -103,6 +108,35 @@ class RelTreeWriterImpl(
       case _ => // ignore
     }
 
+    if (withJoinHint) {
+      rel match {
+        case join: Join =>
+          val joinHints = FlinkHints.getAllJoinHints(join.getHints)
+          if (joinHints.nonEmpty) {
+            printValues.add(Pair.of("joinHints", RelExplainUtil.hintsToString(joinHints)))
+          }
+        case _ => // ignore
+      }
+    }
+
+    if (withQueryBlockAlias) {
+      rel match {
+        case node: Hintable =>
+          node match {
+            case _: TableScan =>
+            // We don't need to pint hints about TableScan because TableScan will always
+            // print hints if exist. See more in such as LogicalTableScan#explainTerms
+            case _ =>
+              val queryBlockAliasHints = FlinkHints.getQueryBlockAliasHints(node.getHints)
+              if (queryBlockAliasHints.nonEmpty) {
+                printValues.add(
+                  Pair.of("hints", RelExplainUtil.hintsToString(queryBlockAliasHints)))
+              }
+          }
+        case _ => // ignore
+      }
+    }
+
     if (!printValues.isEmpty) {
       var j = 0
       printValues.toSeq.foreach {
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java
new file mode 100644
index 00000000000..1239c46448b
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java
@@ -0,0 +1,225 @@
+/*
+ * 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.alias;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.catalog.Catalog;
+import org.apache.flink.table.catalog.CatalogManager;
+import org.apache.flink.table.catalog.GenericInMemoryCatalog;
+import org.apache.flink.table.planner.calcite.FlinkRelBuilder;
+import org.apache.flink.table.planner.hint.FlinkHints;
+import org.apache.flink.table.planner.hint.JoinStrategy;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.utils.BatchTableTestUtil;
+import org.apache.flink.table.planner.utils.PlannerMocks;
+import org.apache.flink.table.planner.utils.TableTestBase;
+import org.apache.flink.table.utils.CatalogManagerMocks;
+
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Collections;
+
+/** A test class for {@link ClearJoinHintWithInvalidPropagationShuttle}. */
+public class ClearJoinHintWithInvalidPropagationShuttleTest extends TableTestBase {
+
+    private final BatchTableTestUtil util = batchTestUtil(TableConfig.getDefault());
+    private final Catalog catalog = new GenericInMemoryCatalog("MockCatalog", "default");
+    private final CatalogManager catalogManager =
+            CatalogManagerMocks.preparedCatalogManager()
+                    .defaultCatalog("builtin", catalog)
+                    .config(
+                            Configuration.fromMap(
+                                    Collections.singletonMap(
+                                            ExecutionOptions.RUNTIME_MODE.key(),
+                                            RuntimeExecutionMode.BATCH.name())))
+                    .build();
+    private final PlannerMocks plannerMocks =
+            PlannerMocks.newBuilder()
+                    .withBatchMode(true)
+                    .withCatalogManager(catalogManager)
+                    .build();
+    private final FlinkRelBuilder builder = plannerMocks.getPlannerContext().createRelBuilder();
+
+    @Before
+    public void before() throws Exception {
+        util.tableEnv().registerCatalog("testCatalog", catalog);
+        util.tableEnv().executeSql("use catalog testCatalog");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE t1 (\n"
+                                + "  a BIGINT\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE t2 (\n"
+                                + "  a BIGINT\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE t3 (\n"
+                                + "  a BIGINT\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+    }
+
+    @Test
+    public void testNoNeedToClearJoinHint() {
+        // SELECT /*+ BROADCAST(t1)*/t1.a FROM t1 JOIN t2 ON t1.a = t2.a
+        RelHint joinHintInView =
+                RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t1").build();
+
+        RelNode root =
+                builder.scan("t1")
+                        .scan("t2")
+                        .join(
+                                JoinRelType.INNER,
+                                builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a")))
+                        .project(builder.field(1, 0, "a"))
+                        .hints(joinHintInView)
+                        .build();
+        verifyRelPlan(root);
+    }
+
+    @Test
+    public void testClearJoinHintWithInvalidPropagationToViewWhileViewHasJoinHints() {
+        //  SELECT /*+ BROADCAST(t3)*/t4.a FROM (
+        //      SELECT /*+ BROADCAST(t1)*/t1.a FROM t1 JOIN t2 ON t1.a = t2.a
+        //  ) t4 JOIN t3 ON t4.a = t3.a
+        RelHint joinHintInView =
+                RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t1").build();
+
+        RelHint joinHintRoot =
+                RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t3").build();
+
+        RelHint aliasHint = RelHint.builder(FlinkHints.HINT_ALIAS).hintOption("t4").build();
+
+        RelNode root =
+                builder.scan("t1")
+                        .scan("t2")
+                        .join(
+                                JoinRelType.INNER,
+                                builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a")))
+                        .project(builder.field(1, 0, "a"))
+                        .hints(joinHintInView, aliasHint)
+                        .scan("t3")
+                        .join(
+                                JoinRelType.INNER,
+                                builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a")))
+                        .project(builder.field(1, 0, "a"))
+                        .hints(joinHintRoot)
+                        .build();
+        verifyRelPlan(root);
+    }
+
+    @Test
+    public void testClearJoinHintWithInvalidPropagationToViewWhileViewHasNoJoinHints() {
+        //  SELECT /*+ BROADCAST(t3)*/t4.a FROM (
+        //      SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a
+        //  ) t4 JOIN t3 ON t4.a = t3.a
+        RelHint joinHintRoot =
+                RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t3").build();
+
+        RelHint aliasHint = RelHint.builder(FlinkHints.HINT_ALIAS).hintOption("t4").build();
+
+        RelNode root =
+                builder.scan("t1")
+                        .scan("t2")
+                        .join(
+                                JoinRelType.INNER,
+                                builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a")))
+                        .project(builder.field(1, 0, "a"))
+                        .hints(aliasHint)
+                        .scan("t3")
+                        .join(
+                                JoinRelType.INNER,
+                                builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a")))
+                        .project(builder.field(1, 0, "a"))
+                        .hints(joinHintRoot)
+                        .build();
+        verifyRelPlan(root);
+    }
+
+    @Test
+    public void testClearJoinHintWithoutPropagatingToView() {
+        //  SELECT /*+ BROADCAST(t1)*/t4.a FROM (
+        //      SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a
+        //  ) t4 JOIN t3 ON t4.a = t3.a
+        RelHint joinHintRoot =
+                RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t1").build();
+
+        RelHint aliasHint = RelHint.builder(FlinkHints.HINT_ALIAS).hintOption("t4").build();
+
+        RelNode root =
+                builder.scan("t1")
+                        .scan("t2")
+                        .join(
+                                JoinRelType.INNER,
+                                builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a")))
+                        .project(builder.field(1, 0, "a"))
+                        .hints(aliasHint)
+                        .scan("t3")
+                        .join(
+                                JoinRelType.INNER,
+                                builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a")))
+                        .project(builder.field(1, 0, "a"))
+                        .hints(joinHintRoot)
+                        .build();
+        verifyRelPlan(root);
+    }
+
+    private String buildRelPlanWithQueryBlockAlias(RelNode node) {
+        return System.lineSeparator()
+                + FlinkRelOptUtil.toString(
+                        node, SqlExplainLevel.EXPPLAN_ATTRIBUTES, false, false, true, false, true);
+    }
+
+    private void verifyRelPlan(RelNode node) {
+        String plan = buildRelPlanWithQueryBlockAlias(node);
+        util.assertEqualsOrExpand("beforePropagatingHints", plan, true);
+
+        RelNode rootAfterHintPropagation = RelOptUtil.propagateRelHints(node, false);
+        plan = buildRelPlanWithQueryBlockAlias(rootAfterHintPropagation);
+        util.assertEqualsOrExpand("afterPropagatingHints", plan, true);
+
+        RelNode rootAfterClearingJoinHintWithInvalidPropagation =
+                rootAfterHintPropagation.accept(new ClearJoinHintWithInvalidPropagationShuttle());
+        plan = buildRelPlanWithQueryBlockAlias(rootAfterClearingJoinHintWithInvalidPropagation);
+        util.assertEqualsOrExpand("afterClearingJoinHints", plan, false);
+    }
+}
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/joinhint/BroadcastJoinHintTest.java
new file mode 100644
index 00000000000..4a96fcd3bbc
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.batch.sql.join.joinhint;
+
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+/** Test for Broadcast join hint. */
+public class BroadcastJoinHintTest extends JoinHintTestBase {
+
+    @Override
+    protected String getTestSingleJoinHint() {
+        return JoinStrategy.BROADCAST.getJoinHintName();
+    }
+
+    @Override
+    protected String getDisabledOperatorName() {
+        return "HashJoin";
+    }
+}
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/joinhint/JoinHintTestBase.java
new file mode 100644
index 00000000000..d19f3547516
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/JoinHintTestBase.java
@@ -0,0 +1,726 @@
+/*
+ * 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.batch.sql.join.joinhint;
+
+import org.apache.flink.table.api.ExplainDetail;
+import org.apache.flink.table.api.SqlParserException;
+import org.apache.flink.table.api.StatementSet;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.planner.hint.JoinStrategy;
+import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder;
+import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil;
+import org.apache.flink.table.planner.utils.BatchTableTestUtil;
+import org.apache.flink.table.planner.utils.PlanKind;
+import org.apache.flink.table.planner.utils.TableTestBase;
+
+import org.apache.flink.shaded.curator5.org.apache.curator.shaded.com.google.common.collect.Lists;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.logging.log4j.util.Strings;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import scala.Enumeration;
+
+import static scala.runtime.BoxedUnit.UNIT;
+
+/**
+ * A test base for join hint.
+ *
+ * <p>TODO add test to cover legacy table source.
+ */
+public abstract class JoinHintTestBase extends TableTestBase {
+
+    protected BatchTableTestUtil util;
+
+    private final List<String> allJoinHintNames =
+            Lists.newArrayList(JoinStrategy.values()).stream()
+                    .map(JoinStrategy::getJoinHintName)
+                    .collect(Collectors.toList());
+
+    @Before
+    public void before() {
+        util = batchTestUtil(TableConfig.getDefault());
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T1 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T2 (\n"
+                                + "  a2 BIGINT,\n"
+                                + "  b2 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE T3 (\n"
+                                + "  a3 BIGINT,\n"
+                                + "  b3 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        util.tableEnv().executeSql("CREATE View V4 as select a3 as a4, b3 as b4 from T3");
+    }
+
+    protected abstract String getTestSingleJoinHint();
+
+    protected abstract String getDisabledOperatorName();
+
+    protected void verifyRelPlanByCustom(String sql) {
+        util.doVerifyPlan(
+                sql,
+                new ExplainDetail[] {},
+                false,
+                new Enumeration.Value[] {PlanKind.AST(), PlanKind.OPT_REL()},
+                true);
+    }
+
+    protected void verifyRelPlanByCustom(StatementSet set) {
+        util.doVerifyPlan(
+                set,
+                new ExplainDetail[] {},
+                false,
+                new Enumeration.Value[] {PlanKind.AST(), PlanKind.OPT_REL()},
+                () -> UNIT,
+                true);
+    }
+
+    protected List<String> getOtherJoinHints() {
+        return allJoinHintNames.stream()
+                .filter(name -> !name.equals(getTestSingleJoinHint()))
+                .collect(Collectors.toList());
+    }
+
+    @Test
+    public void testSimpleJoinHintWithLeftSideAsBuildSide() {
+        String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testSimpleJoinHintWithRightSideAsBuildSide() {
+        String sql = "select /*+ %s(T2) */* from T1 join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithMultiJoinAndFirstSideAsBuildSide1() {
+        // the T1 will be the build side in first join
+        String sql =
+                "select /*+ %s(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithMultiJoinAndFirstSideAsBuildSide2() {
+        String sql =
+                "select /*+ %s(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1() {
+        String sql =
+                "select /*+ %s(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2() {
+        String sql =
+                "select /*+ %s(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides() {
+        String sql =
+                "select /*+ %s(T1, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithUnknownTable() {
+        thrown().expect(ValidationException.class);
+        thrown().expectMessage(
+                        "The options of following hints cannot match the name of input tables or views:");
+        String sql = "select /*+ %s(T99) */* from T1 join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithView() {
+        String sql = "select /*+ %s(V4) */* from T1 join V4 on T1.a1 = V4.a4";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithUnknownView() {
+        thrown().expect(ValidationException.class);
+        thrown().expectMessage(
+                        String.format(
+                                "The options of following hints cannot match the name of input tables or views: \n"
+                                        + "`%s(V99)`",
+                                getTestSingleJoinHint()));
+        String sql = "select /*+ %s(V99) */* from T1 join V4 on T1.a1 = V4.a4";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithEquiPred() {
+        String sql = "select /*+ %s(T1) */* from T1, T2 where T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithEquiPredAndFilter() {
+        String sql = "select /*+ %s(T1) */* from T1, T2 where T1.a1 = T2.a2 and T1.a1 > 1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithEquiAndLocalPred() {
+        String sql = "select /*+ %s(T1) */* from T1 inner join T2 on T1.a1 = T2.a2 and T1.a1 < 1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithEquiAndNonEquiPred() {
+        String sql =
+                "select /*+ %s(T1) */* from T1 inner join T2 on T1.b1 = T2.b2 and T1.a1 < 1 and T1.a1 < T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithoutJoinPred() {
+        String sql = "select /*+ %s(T1) */* from T1, T2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithNonEquiPred() {
+        String sql = "select /*+ %s(T1) */* from T1 inner join T2 on T1.a1 > T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithLeftJoinAndLeftSideAsBuildSide() {
+        String sql = "select /*+ %s(T1) */* from T1 left join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithLeftJoinAndRightSideAsBuildSide() {
+        String sql = "select /*+ %s(T2) */* from T1 left join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithRightJoinAndLeftSideAsBuildSide() {
+        String sql = "select /*+ %s(T1) */* from T1 right join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithRightJoinAndRightSideAsBuildSide() {
+        String sql = "select /*+ %s(T2) */* from T1 right join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithFullJoinAndLeftSideAsBuildSide() {
+        String sql = "select /*+ %s(T1) */* from T1 full join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithFullJoinAndRightSideAsBuildSide() {
+        String sql = "select /*+ %s(T2) */* from T1 full join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    // TODO currently join hint is not supported on SEMI join, it will use default join strategy by
+    // planner
+    @Test
+    public void testJoinHintWithSemiJoinAndLeftSideAsBuildSide() {
+        String sql = "select /*+ %s(T1) */* from T1 where a1 in (select a2 from T2)";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    // TODO currently join hint is not supported on SEMI join, it will use default join strategy by
+    // planner
+    @Test
+    public void testJoinHintWithSemiJoinAndRightSideAsBuildSide() {
+        String sql = "select /*+ %s(T2) */* from T1 where a1 in (select a2 from T2)";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    // TODO currently join hint is not supported on ANTI join, it will use default join strategy by
+    // planner
+    @Test
+    public void testJoinHintWithAntiJoinAndLeftSideAsBuildSide() {
+        String sql = "select /*+ %s(T1) */* from T1 where a1 not in (select a2 from T2)";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    // TODO currently join hint is not supported on ANTI join, it will use default join strategy by
+    // planner
+    @Test
+    public void testJoinHintWithAntiJoinAndRightSideAsBuildSide() {
+        String sql = "select /*+ %s(T2) */* from T1 where a1 not in (select a2 from T2)";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithMultiArgsAndLeftSideFirst() {
+        // the first arg will be chosen as the build side
+        String sql = "select /*+ %s(T1, T2) */* from T1 right join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithMultiArgsAndRightSideFirst() {
+        // the first arg will be chosen as the build side
+        String sql = "select /*+ %s(T2, T1) */* from T1 right join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testMultiJoinHints() {
+        // the first join hint will be chosen
+        String sql = "select /*+ %s(T1), %s */* from T1 join T2 on T1.a1 = T2.a2";
+
+        String otherJoinHints =
+                Strings.join(
+                        getOtherJoinHints().stream()
+                                .map(name -> String.format("%s(T1)", name))
+                                .collect(Collectors.toList()),
+                        ',');
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), otherJoinHints));
+    }
+
+    @Test
+    public void testMultiJoinHintsWithTheFirstOneIsInvalid() {
+        // the first join hint is invalid because it is not equi join except NEST_LOOP
+        String sql = "select /*+ %s(T1), NEST_LOOP(T1) */* from T1 join T2 on T1.a1 > T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithoutAffectingJoinInView() {
+        // the join in V2 will use the planner's default join strategy,
+        // and the join between T1 and V2 will use BROADCAST
+        util.tableEnv()
+                .executeSql("create view V2 as select T1.* from T1 join T2 on T1.a1 = T2.a2");
+
+        String sql = "select /*+ %s(T1)*/T1.* from T1 join V2 on T1.a1 = V2.a1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithoutAffectingJoinInSubQuery() {
+        // the join in sub-query will use the planner's default join strategy,
+        // and the join outside will use BROADCAST
+        String sql =
+                "select /*+ %s(T1)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithTableAlias() {
+        // the join in sub-query will use the planner's default join strategy,
+        // and the join between T1 and alias V2 will use BROADCAST
+        String sql =
+                "select /*+ %s(V2)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintsWithMultiSameJoinHintsAndSingleArg() {
+        // the first join hint will be chosen and T1 will be chosen as the build side
+        String sql = "select /*+ %s(T1), %s(T2) */* from T1 join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintsWithDuplicatedArgs() {
+        // T1 will be chosen as the build side
+        String sql = "select /*+ %s(T1, T1) */* from T1 join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintsWithMultiSameJoinHintsAndMultiArgs() {
+        // the first join hint will be chosen and T1 will be chosen as the build side
+        String sql = "select /*+ %s(T1, T2), %s(T2, T1) */* from T1 join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintsWithMultiHintsThrowException() {
+        thrown().expect(SqlParserException.class);
+        thrown().expectMessage("SQL parse failed.");
+        String sql = "select /*+ %s(T1) */ /*+ %s(T2) */ * from T1 join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithDisabledOperator() {
+        util.tableEnv()
+                .getConfig()
+                .set(
+                        ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
+                        getDisabledOperatorName());
+
+        String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintsWithUnion() {
+        // there are two query blocks and join hints are independent
+        String sql =
+                "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2 union select /*+ %s(T3) */* from T3 join T1 on T3.a3 = T1.a1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintsWithFilter() {
+        // there are two query blocks and join hints are independent
+        String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2 where T1.a1 > 5";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintsWithCalc() {
+        // there are two query blocks and join hints are independent
+        String sql = "select /*+ %s(T1) */a1 + 1, a1 * 10 from T1 join T2 on T1.a1 = T2.a2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintInView() {
+        // the build side in view is left
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2",
+                                getTestSingleJoinHint()));
+
+        // the build side outside is right
+        String sql = "select /*+ %s(V2)*/T3.* from T3 join V2 on T3.a3 = V2.a1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintInMultiLevelView() {
+        // the inside view keeps multi alias
+        // the build side in this view is left
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2",
+                                getTestSingleJoinHint()));
+
+        // the build side in this view is right
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V3 as select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1",
+                                getTestSingleJoinHint()));
+
+        // the build side outside is left
+        String sql = "select /*+ %s(V3)*/V3.* from V3 join T1 on V3.a1 = T1.a1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintsOnSameViewWithoutReusingView() {
+        // the build side in this view is left
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2",
+                                getTestSingleJoinHint()));
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE S1 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE S2 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        StatementSet set = util.tableEnv().createStatementSet();
+
+        // the calc will be pushed down
+        set.addInsertSql(
+                String.format(
+                        "insert into S1 select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 2",
+                        getTestSingleJoinHint()));
+        set.addInsertSql(
+                String.format(
+                        "insert into S2 select /*+ %s(T1)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 5",
+                        getTestSingleJoinHint()));
+
+        verifyRelPlanByCustom(set);
+    }
+
+    @Test
+    public void testJoinHintsOnSameViewWithReusingView() {
+        util.tableEnv()
+                .getConfig()
+                .set(
+                        RelNodeBlockPlanBuilder
+                                .TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED(),
+                        true);
+
+        // the build side in this view is left
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2",
+                                getTestSingleJoinHint()));
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE S1 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE S2 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        StatementSet set = util.tableEnv().createStatementSet();
+
+        // the calc will be pushed down because the view has same digest
+        set.addInsertSql(
+                String.format(
+                        "insert into S1 select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 2",
+                        getTestSingleJoinHint()));
+        set.addInsertSql(
+                String.format(
+                        "insert into S2 select /*+ %s(T1)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 5",
+                        getTestSingleJoinHint()));
+
+        verifyRelPlanByCustom(set);
+    }
+
+    @Test
+    public void testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints() {
+        util.tableEnv()
+                .getConfig()
+                .set(
+                        RelNodeBlockPlanBuilder
+                                .TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED(),
+                        true);
+
+        // the build side in this view is left
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2",
+                                getTestSingleJoinHint()));
+
+        // the build side in this view is left
+        // V2 and V3 have different join hints
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V3 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2",
+                                getOtherJoinHints().get(0)));
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE S1 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        util.tableEnv()
+                .executeSql(
+                        "CREATE TABLE S2 (\n"
+                                + "  a1 BIGINT,\n"
+                                + "  b1 VARCHAR\n"
+                                + ") WITH (\n"
+                                + " 'connector' = 'values',\n"
+                                + " 'bounded' = 'true'\n"
+                                + ")");
+
+        StatementSet set = util.tableEnv().createStatementSet();
+
+        // the calc will not be pushed down because the view has different digest
+        set.addInsertSql(
+                String.format(
+                        "insert into S1 select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 2",
+                        getTestSingleJoinHint()));
+        set.addInsertSql(
+                String.format(
+                        "insert into S2 select /*+ %s(T1)*/ T1.* from T1 join V3 on T1.a1 = V3.a1 where V3.a1 > 5",
+                        getOtherJoinHints().get(0)));
+
+        verifyRelPlanByCustom(set);
+    }
+
+    @Test
+    public void testJoinHintWithSubStringViewName1() {
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2",
+                                getTestSingleJoinHint()));
+
+        // the build side in this view is right
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V22 as select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1",
+                                getTestSingleJoinHint()));
+
+        // the build side outside is left
+        String sql = "select /*+ %s(V22)*/V22.* from V22 join T1 on V22.a1 = T1.a1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithSubStringViewName2() {
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V22 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2",
+                                getTestSingleJoinHint()));
+
+        // the build side in this view is right
+        util.tableEnv()
+                .executeSql(
+                        String.format(
+                                "create view V2 as select /*+ %s(V22)*/ T1.* from T1 join V22 on T1.a1 = V22.a1",
+                                getTestSingleJoinHint()));
+
+        // the build side outside is left
+        String sql = "select /*+ %s(V2)*/V2.* from V2 join T1 on V2.a1 = T1.a1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    protected String buildAstPlanWithQueryBlockAlias(List<RelNode> relNodes) {
+        StringBuilder astBuilder = new StringBuilder();
+        relNodes.forEach(
+                node ->
+                        astBuilder
+                                .append(System.lineSeparator())
+                                .append(
+                                        FlinkRelOptUtil.toString(
+                                                node,
+                                                SqlExplainLevel.EXPPLAN_ATTRIBUTES,
+                                                false,
+                                                false,
+                                                true,
+                                                false,
+                                                true)));
+        return astBuilder.toString();
+    }
+}
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/joinhint/NestLoopJoinHintTest.java
new file mode 100644
index 00000000000..5c8b1e8f3e8
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.batch.sql.join.joinhint;
+
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+/** Test for Nest Loop join hint. */
+public class NestLoopJoinHintTest extends JoinHintTestBase {
+
+    @Override
+    protected String getTestSingleJoinHint() {
+        return JoinStrategy.NEST_LOOP.getJoinHintName();
+    }
+
+    @Override
+    protected String getDisabledOperatorName() {
+        return "NestedLoopJoin";
+    }
+}
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/joinhint/ShuffleHashJoinHintTest.java
new file mode 100644
index 00000000000..bd19dcc53eb
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.batch.sql.join.joinhint;
+
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+/** Test for Shuffle Hash join hint. */
+public class ShuffleHashJoinHintTest extends JoinHintTestBase {
+
+    @Override
+    protected String getTestSingleJoinHint() {
+        return JoinStrategy.SHUFFLE_HASH.getJoinHintName();
+    }
+
+    @Override
+    protected String getDisabledOperatorName() {
+        return "HashJoin";
+    }
+}
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/joinhint/ShuffleMergeJoinHintTest.java
new file mode 100644
index 00000000000..0cb31812184
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.java
@@ -0,0 +1,35 @@
+/*
+ * 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.batch.sql.join.joinhint;
+
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+/** Test for Shuffle Merge join hint. */
+public class ShuffleMergeJoinHintTest extends JoinHintTestBase {
+
+    @Override
+    protected String getTestSingleJoinHint() {
+        return JoinStrategy.SHUFFLE_MERGE.getJoinHintName();
+    }
+
+    @Override
+    protected String getDisabledOperatorName() {
+        return "SortMergeJoin";
+    }
+}
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
new file mode 100644
index 00000000000..6b6d1a3fad6
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.optimize;
+
+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.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+
+import org.apache.calcite.rel.RelNode;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** A test class for {@link ClearQueryBlockAliasResolver}. */
+public class ClearQueryBlockAliasResolverTest extends JoinHintTestBase {
+
+    // use any join hint for test
+    @Override
+    protected String getTestSingleJoinHint() {
+        return JoinStrategy.BROADCAST.getJoinHintName();
+    }
+
+    @Override
+    protected String getDisabledOperatorName() {
+        return "HashJoin";
+    }
+
+    /**
+     * Customize logic to verify the RelNode tree by sql.
+     *
+     * <p>Currently, mainly copy from {@link TableTestBase} and customize it.
+     */
+    @Override
+    protected void verifyRelPlanByCustom(String sql) {
+        Table table = util.tableEnv().sqlQuery(sql);
+        RelNode relNode = TableTestUtil.toRelNode(table);
+        verifyRelPlanAfterResolverWithSql(sql, Collections.singletonList(relNode));
+    }
+
+    /**
+     * Customize logic to verify the RelNode tree by StatementSet.
+     *
+     * <p>Currently, mainly copy from {@link TableTestBase} and customize it.
+     */
+    @Override
+    protected void verifyRelPlanByCustom(StatementSet set) {
+        StatementSetImpl<?> testStmtSet = (StatementSetImpl<?>) set;
+
+        List<RelNode> relNodes =
+                testStmtSet.getOperations().stream()
+                        .map(node -> util.getPlanner().translateToRel(node))
+                        .collect(Collectors.toList());
+        verifyRelPlanAfterResolverWithStatementSet(relNodes);
+    }
+
+    /**
+     * Customize logic to verify the RelNode tree.
+     *
+     * <p>Currently, mainly copy from {@link TableTestBase} and customize it.
+     */
+    private void verifyRelPlanAfterResolverWithSql(String sql, List<RelNode> relNodes) {
+        relNodes = clearQueryBlockAlias(relNodes);
+        String astPlan = buildAstPlanWithQueryBlockAlias(relNodes);
+
+        util.assertEqualsOrExpand("sql", sql, true);
+        util.assertEqualsOrExpand("ast", astPlan, false);
+    }
+
+    private void verifyRelPlanAfterResolverWithStatementSet(List<RelNode> relNodes) {
+        relNodes = clearQueryBlockAlias(relNodes);
+        String astPlan = buildAstPlanWithQueryBlockAlias(relNodes);
+
+        util.assertEqualsOrExpand("ast", astPlan, false);
+    }
+
+    private List<RelNode> clearQueryBlockAlias(List<RelNode> relNodes) {
+        JoinHintResolver joinHintResolver = new JoinHintResolver();
+        relNodes = joinHintResolver.resolve(relNodes);
+        ClearQueryBlockAliasResolver clearQueryBlockAliasResolver =
+                new ClearQueryBlockAliasResolver();
+        return clearQueryBlockAliasResolver.resolve(relNodes);
+    }
+}
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
new file mode 100644
index 00000000000..bc458317c02
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java
@@ -0,0 +1,101 @@
+/*
+ * 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.optimize;
+
+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.utils.TableTestBase;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+
+import org.apache.calcite.rel.RelNode;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/** A test class for {@link JoinHintResolver}. */
+public class JoinHintResolverTest extends JoinHintTestBase {
+
+    // use any join hint for test
+    @Override
+    protected String getTestSingleJoinHint() {
+        return JoinStrategy.BROADCAST.getJoinHintName();
+    }
+
+    @Override
+    protected String getDisabledOperatorName() {
+        return "HashJoin";
+    }
+
+    /**
+     * Customize logic to verify the RelNode tree by sql.
+     *
+     * <p>Currently, mainly copy from {@link TableTestBase} and customize it.
+     */
+    @Override
+    protected void verifyRelPlanByCustom(String sql) {
+        Table table = util.tableEnv().sqlQuery(sql);
+        RelNode relNode = TableTestUtil.toRelNode(table);
+        verifyRelPlanAfterResolverWithSql(sql, Collections.singletonList(relNode));
+    }
+
+    /**
+     * Customize logic to verify the RelNode tree by StatementSet.
+     *
+     * <p>Currently, mainly copy from {@link TableTestBase} and customize it.
+     */
+    @Override
+    protected void verifyRelPlanByCustom(StatementSet set) {
+        StatementSetImpl<?> testStmtSet = (StatementSetImpl<?>) set;
+
+        List<RelNode> relNodes =
+                testStmtSet.getOperations().stream()
+                        .map(node -> util.getPlanner().translateToRel(node))
+                        .collect(Collectors.toList());
+        verifyRelPlanAfterResolverWithStatementSet(relNodes);
+    }
+
+    /**
+     * Customize logic to verify the RelNode tree.
+     *
+     * <p>Currently, mainly copy from {@link TableTestBase} and customize it.
+     */
+    private void verifyRelPlanAfterResolverWithSql(String sql, List<RelNode> relNodes) {
+        relNodes = resolveJoinHint(relNodes);
+        String astPlan = buildAstPlanWithQueryBlockAlias(relNodes);
+
+        util.assertEqualsOrExpand("sql", sql, true);
+        util.assertEqualsOrExpand("ast", astPlan, false);
+    }
+
+    private void verifyRelPlanAfterResolverWithStatementSet(List<RelNode> relNodes) {
+        relNodes = resolveJoinHint(relNodes);
+        String astPlan = buildAstPlanWithQueryBlockAlias(relNodes);
+
+        util.assertEqualsOrExpand("ast", astPlan, false);
+    }
+
+    private List<RelNode> resolveJoinHint(List<RelNode> relNodes) {
+        JoinHintResolver joinHintResolver = new JoinHintResolver();
+        return joinHintResolver.resolve(relNodes);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.xml
new file mode 100644
index 00000000000..a5d50095e04
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.xml
@@ -0,0 +1,150 @@
+<?xml version="1.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.
+-->
+<Root>
+  <TestCase name="testClearJoinHintWithInvalidPropagationToViewWhileViewHasJoinHints">
+    <Resource name="beforePropagatingHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalProject(a=[$0], hints=[[[ALIAS options:[t4]]]]), rowType=[RecordType(BIGINT a)]
+   :  +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :     :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   :     +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t3]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+    <Resource name="afterPropagatingHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t3]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalProject(a=[$0], hints=[[[ALIAS options:[t4]]]]), rowType=[RecordType(BIGINT a)]
+   :  +- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t1]][BROADCAST inheritPath:[0, 0, 0] options:[t3]]]], hints=[[[ALIAS inheritPath:[0] options:[t4]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :     :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   :     +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t3]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+    <Resource name="afterClearingJoinHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t3]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalProject(a=[$0], hints=[[[ALIAS options:[t4]]]]), rowType=[RecordType(BIGINT a)]
+   :  +- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t1]]]], hints=[[[ALIAS inheritPath:[0] options:[t4]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :     :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   :     +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t3]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testClearJoinHintWithInvalidPropagationToViewWhileViewHasNoJoinHints">
+    <Resource name="beforePropagatingHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalProject(a=[$0], hints=[[[ALIAS options:[t4]]]]), rowType=[RecordType(BIGINT a)]
+   :  +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :     :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   :     +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t3]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+    <Resource name="afterPropagatingHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t3]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalProject(a=[$0], hints=[[[ALIAS options:[t4]]]]), rowType=[RecordType(BIGINT a)]
+   :  +- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0, 0] options:[t3]]]], hints=[[[ALIAS inheritPath:[0] options:[t4]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :     :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   :     +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t3]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+    <Resource name="afterClearingJoinHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t3]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalProject(a=[$0], hints=[[[ALIAS options:[t4]]]]), rowType=[RecordType(BIGINT a)]
+   :  +- LogicalJoin(condition=[=($0, $1)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[t4]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :     :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   :     +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t3]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testClearJoinHintWithoutPropagatingToView">
+    <Resource name="beforePropagatingHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalProject(a=[$0], hints=[[[ALIAS options:[t4]]]]), rowType=[RecordType(BIGINT a)]
+   :  +- LogicalJoin(condition=[=($0, $1)], joinType=[inner]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :     :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   :     +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t3]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+    <Resource name="afterPropagatingHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t1]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalProject(a=[$0], hints=[[[ALIAS options:[t4]]]]), rowType=[RecordType(BIGINT a)]
+   :  +- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0, 0] options:[t1]]]], hints=[[[ALIAS inheritPath:[0] options:[t4]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :     :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   :     +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t3]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+    <Resource name="afterClearingJoinHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t1]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalProject(a=[$0], hints=[[[ALIAS options:[t4]]]]), rowType=[RecordType(BIGINT a)]
+   :  +- LogicalJoin(condition=[=($0, $1)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[t4]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :     :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   :     +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t3]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testNoNeedToClearJoinHint">
+    <Resource name="beforePropagatingHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+    <Resource name="afterPropagatingHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t1]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+    <Resource name="afterClearingJoinHints">
+      <![CDATA[
+LogicalProject(a=[$0]), rowType=[RecordType(BIGINT a)]
++- LogicalJoin(condition=[=($0, $1)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[t1]]]]), rowType=[RecordType(BIGINT a, BIGINT a0)]
+   :- LogicalTableScan(table=[[builtin, default, t1]]), rowType=[RecordType(BIGINT a)]
+   +- LogicalTableScan(table=[[builtin, default, t2]]), rowType=[RecordType(BIGINT a)]
+]]>
+    </Resource>
+  </TestCase>
+</Root>
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 f8fc67e6457..8fcc68cdde5 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
@@ -432,6 +432,38 @@ Calc(select=[c, 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSelfJoin">
+    <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)
+         ]]>
+    </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)]]])
+]]>
+    </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])
 ]]>
     </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/joinhint/BroadcastJoinHintTest.xml
new file mode 100644
index 00000000000..dc60bb7e784
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.xml
@@ -0,0 +1,1210 @@
+<?xml version="1.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.
+-->
+<Root>
+  <TestCase name="testJoinHintInMultiLevelView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V3)*/V3.* from V3 join T1 on V3.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[V3]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[V2]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V3]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- 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])
+   :        +- Exchange(distribution=[broadcast])
+   :           +- Calc(select=[a1])
+   :              +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
+   :                 :- Exchange(distribution=[broadcast])
+   :                 :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V2)*/T3.* from T3 join V2 on T3.a3 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a3=[$0], b3=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[V2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a3, b3])
++- HashJoin(joinType=[InnerJoin], where=[=(a3, a1)], select=[a3, b3, a1], isBroadcast=[true], build=[right])
+   :- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+   +- Exchange(distribution=[broadcast])
+      +- Calc(select=[a1])
+         +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
+            :- Exchange(distribution=[broadcast])
+            :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- 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])
+
+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])
+         +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- 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])
+                  +- 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])
+         +- 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])
+                  +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- Exchange(distribution=[broadcast])
+         +- Calc(select=[a1], where=[>(a1, 2)])
+            +- Calc(select=[a1, b1])
+               +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], isBroadcast=[true], build=[left])
+                  :- Exchange(distribution=[broadcast])
+                  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+                  +- TableSourceScan(table=[[default_catalog, default_database, T2, 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], where=[>(a1, 5)])
+         +- Calc(select=[a1, b1])
+            +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], isBroadcast=[true], build=[left])
+               :- Exchange(distribution=[broadcast])
+               :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithCalc">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */a1 + 1, a1 * 10 from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(EXPR$0=[+($0, 1)], EXPR$1=[*($0, 10)])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[+(a1, 1) AS EXPR$0, *(a1, 10) AS EXPR$1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithDuplicatedArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2 where T1.a1 > 5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[>($0, 5)])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndMultiArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2), BROADCAST(T2, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1, T2]][BROADCAST inheritPath:[0] options:[T2, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndSingleArg">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1), BROADCAST(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]][BROADCAST inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithUnion">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2 union select /*+ BROADCAST(T3) */* from T3 join T1 on T3.a3 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalUnion(all=[false])
+:- LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
+:  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+:     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+:     +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
++- LogicalProject(a3=[$0], b3=[$1], a1=[$2], b1=[$3])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T3]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashAggregate(isMerge=[true], groupBy=[a1, b1, a2, b2], select=[a1, b1, a2, b2])
++- Exchange(distribution=[hash[a1, b1, a2, b2]])
+   +- LocalHashAggregate(groupBy=[a1, b1, a2, b2], select=[a1, b1, a2, b2])
+      +- Union(all=[true], union=[a1, b1, a2, b2])
+         :- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+         :  :- Exchange(distribution=[broadcast])
+         :  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+         +- HashJoin(joinType=[InnerJoin], where=[=(a3, a1)], select=[a3, b3, a1, b1], isBroadcast=[true], build=[left])
+            :- Exchange(distribution=[broadcast])
+            :  +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+            +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS NULL(a1), IS NULL(a2), =(a1, a2))], select=[a1, b1], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS NULL(a1), IS NULL(a2), =(a1, a2))], select=[a1, b1], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithDisabledOperator">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndLocalPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 inner join T2 on T1.a1 = T2.a2 and T1.a1 < 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), <($0, 1))], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 inner join T2 on T1.b1 = T2.b2 and T1.a1 < 1 and T1.a1 < T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($1, $3), <($0, 1), <($0, $2))], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[AND(=(b1, b2), <(a1, a2))], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1, T2 where T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[=($0, $2)])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPredAndFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1, T2 where T1.a1 = T2.a2 and T1.a1 > 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), >($0, 1))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[FullOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[BROADCAST inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[FullOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[BROADCAST inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndLeftSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST inheritPath:[0] options:[T1, T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndRightSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2, T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST inheritPath:[0] options:[T2, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide1">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1, T2]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0, 0] options:[T1, T2]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, a2, b2, a3, b3], build=[left])
+:- Exchange(distribution=[hash[b1]])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:     :- Exchange(distribution=[broadcast])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide2">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1, T2]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0, 0] options:[T1, T2]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], build=[left])
+:- Exchange(distribution=[hash[b2]])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:     :- Exchange(distribution=[broadcast])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0, 0] options:[T1, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], isBroadcast=[true], build=[right])
+:- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:  :- Exchange(distribution=[broadcast])
+:  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T2, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0, 0] options:[T2, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, a2, b2, a3, b3], isBroadcast=[true], build=[right])
+:- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[right])
+:  :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:  +- Exchange(distribution=[broadcast])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T2, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0, 0] options:[T2, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], isBroadcast=[true], build=[right])
+:- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[right])
+:  :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:  +- Exchange(distribution=[broadcast])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 inner join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutJoinPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1, T2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], select=[a1, b1], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], select=[a1, b1], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName1">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V22)*/V22.* from V22 join T1 on V22.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[V22]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[V2]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V22]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- 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])
+   :        +- Exchange(distribution=[broadcast])
+   :           +- Calc(select=[a1])
+   :              +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
+   :                 :- Exchange(distribution=[broadcast])
+   :                 :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName2">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V2)*/V2.* from V2 join T1 on V2.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[V2]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[V22]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]][ALIAS inheritPath:[0, 1, 0] options:[V2]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- 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])
+   :        +- Exchange(distribution=[broadcast])
+   :           +- Calc(select=[a1])
+   :              +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left])
+   :                 :- Exchange(distribution=[broadcast])
+   :                 :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHints">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1), SHUFFLE_HASH(T1),SHUFFLE_MERGE(T1),NEST_LOOP(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]][SHUFFLE_HASH inheritPath:[0] options:[T1]][SHUFFLE_MERGE inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithTableAlias">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V2)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[V2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+   +- Exchange(distribution=[broadcast])
+      +- Calc(select=[a1])
+         +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+            :- Exchange(distribution=[hash[a1]])
+            :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+            +- Exchange(distribution=[hash[a2]])
+               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V4) */* from T1 join V4 on T1.a1 = V4.a4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a4=[$2], b4=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[V4]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a4=[$0], b4=[$1], hints=[[[ALIAS options:[V4]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a1, b1, a3, b3], isBroadcast=[true], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHintsWithTheFirstOneIsInvalid">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1), NEST_LOOP(T1) */* from T1 join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], isBroadcast=[true], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
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/joinhint/NestLoopJoinHintTest.xml
new file mode 100644
index 00000000000..3dc576a0029
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.xml
@@ -0,0 +1,1206 @@
+<?xml version="1.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.
+-->
+<Root>
+  <TestCase name="testJoinHintInMultiLevelView">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(V3)*/V3.* from V3 join T1 on V3.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[V3]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[V2]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V3]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- 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])
+   :        +- Exchange(distribution=[broadcast])
+   :           +- Calc(select=[a1])
+   :              +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+   :                 :- Exchange(distribution=[broadcast])
+   :                 :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(V2)*/T3.* from T3 join V2 on T3.a3 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a3=[$0], b3=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[V2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a3, b3])
++- NestedLoopJoin(joinType=[InnerJoin], where=[=(a3, a1)], select=[a3, b3, a1], build=[right])
+   :- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+   +- Exchange(distribution=[broadcast])
+      +- Calc(select=[a1])
+         +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+            :- Exchange(distribution=[broadcast])
+            :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- 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])
+
+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])
+         +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- 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])
+                  +- 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])
+         +- 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])
+               +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- Exchange(distribution=[broadcast])
+         +- Calc(select=[a1], where=[>(a1, 2)])
+            +- Calc(select=[a1, b1])
+               +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], build=[left])
+                  :- Exchange(distribution=[broadcast])
+                  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+                  +- TableSourceScan(table=[[default_catalog, default_database, T2, 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], where=[>(a1, 5)])
+         +- Calc(select=[a1, b1])
+            +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], build=[left])
+               :- Exchange(distribution=[broadcast])
+               :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithCalc">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */a1 + 1, a1 * 10 from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(EXPR$0=[+($0, 1)], EXPR$1=[*($0, 10)])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[+(a1, 1) AS EXPR$0, *(a1, 10) AS EXPR$1])
++- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithDuplicatedArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 join T2 on T1.a1 = T2.a2 where T1.a1 > 5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[>($0, 5)])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndMultiArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1, T2), NEST_LOOP(T2, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1, T2]][NEST_LOOP inheritPath:[0] options:[T2, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndSingleArg">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1), NEST_LOOP(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithUnion">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 join T2 on T1.a1 = T2.a2 union select /*+ NEST_LOOP(T3) */* from T3 join T1 on T3.a3 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalUnion(all=[false])
+:- LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
+:  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+:     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+:     +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
++- LogicalProject(a3=[$0], b3=[$1], a1=[$2], b1=[$3])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T3]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashAggregate(isMerge=[true], groupBy=[a1, b1, a2, b2], select=[a1, b1, a2, b2])
++- Exchange(distribution=[hash[a1, b1, a2, b2]])
+   +- LocalHashAggregate(groupBy=[a1, b1, a2, b2], select=[a1, b1, a2, b2])
+      +- Union(all=[true], union=[a1, b1, a2, b2])
+         :- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+         :  :- Exchange(distribution=[broadcast])
+         :  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+         +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a3, a1)], select=[a3, b3, a1, b1], build=[left])
+            :- Exchange(distribution=[broadcast])
+            :  +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+            +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS NULL(a1), IS NULL(a2), =(a1, a2))], select=[a1, b1], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T2) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS NULL(a1), IS NULL(a2), =(a1, a2))], select=[a1, b1], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithDisabledOperator">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndLocalPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 inner join T2 on T1.a1 = T2.a2 and T1.a1 < 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), <($0, 1))], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 inner join T2 on T1.b1 = T2.b2 and T1.a1 < 1 and T1.a1 < T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($1, $3), <($0, 1), <($0, $2))], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[AND(=(b1, b2), <(a1, a2))], select=[a1, b1, a2, b2], build=[left])
+:- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1, T2 where T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[=($0, $2)])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPredAndFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1, T2 where T1.a1 = T2.a2 and T1.a1 > 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), >($0, 1))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[FullOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[single])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[single])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T2) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[FullOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[single])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[single])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T2) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndLeftSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1, T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1, T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndRightSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T2, T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T2, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide1">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1, T2]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0, 0] options:[T1, T2]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, a2, b2, a3, b3], build=[left])
+:- Exchange(distribution=[hash[b1]])
+:  +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:     :- Exchange(distribution=[broadcast])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide2">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1, T2]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0, 0] options:[T1, T2]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], build=[left])
+:- Exchange(distribution=[hash[b2]])
+:  +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:     :- Exchange(distribution=[broadcast])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0, 0] options:[T1, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], build=[right])
+:- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:  :- Exchange(distribution=[broadcast])
+:  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T2, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0, 0] options:[T2, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, a2, b2, a3, b3], build=[right])
+:- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:  :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:  +- Exchange(distribution=[broadcast])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T2, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0, 0] options:[T2, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], build=[right])
+:- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:  :- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:  +- Exchange(distribution=[broadcast])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 inner join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/T1.* from T1 join V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutJoinPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1, T2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], select=[a1, b1], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T2) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], select=[a1, b1], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName1">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(V22)*/V22.* from V22 join T1 on V22.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[V22]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[V2]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V22]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- 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])
+   :        +- Exchange(distribution=[broadcast])
+   :           +- Calc(select=[a1])
+   :              +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+   :                 :- Exchange(distribution=[broadcast])
+   :                 :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName2">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(V2)*/V2.* from V2 join T1 on V2.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[V2]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[V22]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]][ALIAS inheritPath:[0, 1, 0] options:[V2]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- 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])
+   :        +- Exchange(distribution=[broadcast])
+   :           +- Calc(select=[a1])
+   :              +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+   :                 :- Exchange(distribution=[broadcast])
+   :                 :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :                 +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHints">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1), BROADCAST(T1),SHUFFLE_HASH(T1),SHUFFLE_MERGE(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]][BROADCAST inheritPath:[0] options:[T1]][SHUFFLE_HASH inheritPath:[0] options:[T1]][SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithTableAlias">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(V2)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[V2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+   +- Exchange(distribution=[broadcast])
+      +- Calc(select=[a1])
+         +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+            :- Exchange(distribution=[hash[a1]])
+            :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+            +- Exchange(distribution=[hash[a2]])
+               +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithView">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(V4) */* from T1 join V4 on T1.a1 = V4.a4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a4=[$2], b4=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[V4]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a4=[$0], b4=[$1], hints=[[[ALIAS options:[V4]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a1, b1, a3, b3], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHintsWithTheFirstOneIsInvalid">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1), NEST_LOOP(T1) */* from T1 join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
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/joinhint/ShuffleHashJoinHintTest.xml
new file mode 100644
index 00000000000..67f9ad612d4
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.xml
@@ -0,0 +1,1246 @@
+<?xml version="1.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.
+-->
+<Root>
+  <TestCase name="testJoinHintInMultiLevelView">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(V3)*/V3.* from V3 join T1 on V3.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[V3]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[V2]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V3]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
+   :- 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])
+   :        +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+   :           :- Exchange(distribution=[hash[a1]])
+   :           :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :           +- Exchange(distribution=[hash[a2]])
+   :              +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- Exchange(distribution=[hash[a1]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(V2)*/T3.* from T3 join V2 on T3.a3 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a3=[$0], b3=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[V2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a3, b3])
++- HashJoin(joinType=[InnerJoin], where=[=(a3, a1)], select=[a3, b3, a1], build=[right])
+   :- Exchange(distribution=[hash[a3]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+   +- Calc(select=[a1])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+         +- 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])
+
+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])
+         +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+         +- 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])
+                  +- 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])
+         +- 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])
+               +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- Exchange(distribution=[hash[a1]])
+         +- Calc(select=[a1], where=[>(a1, 2)])
+            +- Calc(select=[a1, b1])
+               +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], build=[left])
+                  :- Exchange(distribution=[hash[a1]])
+                  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+                  +- Exchange(distribution=[hash[a2]])
+                     +- TableSourceScan(table=[[default_catalog, default_database, T2, 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])
+      +- Exchange(distribution=[hash[a1]])
+         +- Calc(select=[a1], where=[>(a1, 5)])
+            +- Calc(select=[a1, b1])
+               +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], build=[left])
+                  :- Exchange(distribution=[hash[a1]])
+                  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+                  +- Exchange(distribution=[hash[a2]])
+                     +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithCalc">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */a1 + 1, a1 * 10 from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(EXPR$0=[+($0, 1)], EXPR$1=[*($0, 10)])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[+(a1, 1) AS EXPR$0, *(a1, 10) AS EXPR$1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithDuplicatedArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 join T2 on T1.a1 = T2.a2 where T1.a1 > 5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[>($0, 5)])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndMultiArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1, T2), SHUFFLE_HASH(T2, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1, T2]][SHUFFLE_HASH inheritPath:[0] options:[T2, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndSingleArg">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1), SHUFFLE_HASH(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]][SHUFFLE_HASH inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithUnion">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 join T2 on T1.a1 = T2.a2 union select /*+ SHUFFLE_HASH(T3) */* from T3 join T1 on T3.a3 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalUnion(all=[false])
+:- LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
+:  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+:     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+:     +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
++- LogicalProject(a3=[$0], b3=[$1], a1=[$2], b1=[$3])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T3]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashAggregate(isMerge=[true], groupBy=[a1, b1, a2, b2], select=[a1, b1, a2, b2])
++- Exchange(distribution=[hash[a1, b1, a2, b2]])
+   +- LocalHashAggregate(groupBy=[a1, b1, a2, b2], select=[a1, b1, a2, b2])
+      +- Union(all=[true], union=[a1, b1, a2, b2])
+         :- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+         :  :- Exchange(distribution=[hash[a1]])
+         :  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+         :  +- Exchange(distribution=[hash[a2]])
+         :     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+         +- HashJoin(joinType=[InnerJoin], where=[=(a3, a1)], select=[a3, b3, a1, b1], build=[left])
+            :- Exchange(distribution=[hash[a3]])
+            :  +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+            +- Exchange(distribution=[hash[a1]])
+               +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS NULL(a1), IS NULL(a2), =(a1, a2))], select=[a1, b1], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T2) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS NULL(a1), IS NULL(a2), =(a1, a2))], select=[a1, b1], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithDisabledOperator">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndLocalPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 inner join T2 on T1.a1 = T2.a2 and T1.a1 < 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), <($0, 1))], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 inner join T2 on T1.b1 = T2.b2 and T1.a1 < 1 and T1.a1 < T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($1, $3), <($0, 1), <($0, $2))], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[AND(=(b1, b2), <(a1, a2))], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[b1]])
+:  +- Calc(select=[a1, b1], where=[<(a1, 1)])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
++- Exchange(distribution=[hash[b2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1, T2 where T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[=($0, $2)])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPredAndFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1, T2 where T1.a1 = T2.a2 and T1.a1 > 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), >($0, 1))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[FullOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T2) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[FullOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T2) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndLeftSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1, T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1, T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndRightSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T2, T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T2, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide1">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1, T2]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0, 0] options:[T1, T2]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, a2, b2, a3, b3], build=[left])
+:- Exchange(distribution=[hash[b1]])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide2">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1, T2]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0, 0] options:[T1, T2]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], build=[left])
+:- Exchange(distribution=[hash[b2]])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0, 0] options:[T1, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], build=[right])
+:- Exchange(distribution=[hash[b2]])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T2, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0, 0] options:[T2, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, a2, b2, a3, b3], build=[right])
+:- Exchange(distribution=[hash[b1]])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T2, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0, 0] options:[T2, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], build=[right])
+:- Exchange(distribution=[hash[b2]])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 inner join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/T1.* from T1 join V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutJoinPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1, T2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], select=[a1, b1], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T2) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], select=[a1, b1], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName1">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(V22)*/V22.* from V22 join T1 on V22.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[V22]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[V2]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V22]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
+   :- 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])
+   :        +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+   :           :- Exchange(distribution=[hash[a1]])
+   :           :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :           +- Exchange(distribution=[hash[a2]])
+   :              +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- Exchange(distribution=[hash[a1]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName2">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(V2)*/V2.* from V2 join T1 on V2.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[V2]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[V22]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]][ALIAS inheritPath:[0, 1, 0] options:[V2]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left])
+   :- 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])
+   :        +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left])
+   :           :- Exchange(distribution=[hash[a1]])
+   :           :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :           +- Exchange(distribution=[hash[a2]])
+   :              +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- Exchange(distribution=[hash[a1]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHints">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1), BROADCAST(T1),SHUFFLE_MERGE(T1),NEST_LOOP(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]][BROADCAST inheritPath:[0] options:[T1]][SHUFFLE_MERGE inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithTableAlias">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(V2)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[V2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithView">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(V4) */* from T1 join V4 on T1.a1 = V4.a4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a4=[$2], b4=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[V4]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a4=[$0], b4=[$1], hints=[[[ALIAS options:[V4]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a1, b1, a3, b3], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHintsWithTheFirstOneIsInvalid">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1), NEST_LOOP(T1) */* from T1 join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
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/joinhint/ShuffleMergeJoinHintTest.xml
new file mode 100644
index 00000000000..874d00d39ea
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.xml
@@ -0,0 +1,1246 @@
+<?xml version="1.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.
+-->
+<Root>
+  <TestCase name="testJoinHintInMultiLevelView">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(V3)*/V3.* from V3 join T1 on V3.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[V3]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[V2]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V3]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10])
+   :- 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])
+   :        +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2])
+   :           :- Exchange(distribution=[hash[a1]])
+   :           :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :           +- Exchange(distribution=[hash[a2]])
+   :              +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- Exchange(distribution=[hash[a1]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(V2)*/T3.* from T3 join V2 on T3.a3 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a3=[$0], b3=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[V2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a3, b3])
++- SortMergeJoin(joinType=[InnerJoin], where=[=(a3, a1)], select=[a3, b3, a1])
+   :- Exchange(distribution=[hash[a3]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+   +- Calc(select=[a1])
+      +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+         +- 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])
+
+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])
+         +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+         +- 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])
+                  +- 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])
+         +- 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])
+               +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 2)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1])
++- LogicalProject(a1=[$0], b1=[$1])
+   +- LogicalFilter(condition=[>($2, 5)])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- Exchange(distribution=[hash[a1]])
+         +- Calc(select=[a1], where=[>(a1, 2)])
+            +- Calc(select=[a1, b1])
+               +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2])
+                  :- Exchange(distribution=[hash[a1]])
+                  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+                  +- Exchange(distribution=[hash[a2]])
+                     +- TableSourceScan(table=[[default_catalog, default_database, T2, 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])
+      +- Exchange(distribution=[hash[a1]])
+         +- Calc(select=[a1], where=[>(a1, 5)])
+            +- Calc(select=[a1, b1])
+               +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2])
+                  :- Exchange(distribution=[hash[a1]])
+                  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+                  +- Exchange(distribution=[hash[a2]])
+                     +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithCalc">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */a1 + 1, a1 * 10 from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(EXPR$0=[+($0, 1)], EXPR$1=[*($0, 10)])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[+(a1, 1) AS EXPR$0, *(a1, 10) AS EXPR$1])
++- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithDuplicatedArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 join T2 on T1.a1 = T2.a2 where T1.a1 > 5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[>($0, 5)])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndMultiArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1, T2), SHUFFLE_MERGE(T2, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1, T2]][SHUFFLE_MERGE inheritPath:[0] options:[T2, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndSingleArg">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1), SHUFFLE_MERGE(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]][SHUFFLE_MERGE inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithUnion">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 join T2 on T1.a1 = T2.a2 union select /*+ SHUFFLE_MERGE(T3) */* from T3 join T1 on T3.a3 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalUnion(all=[false])
+:- LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
+:  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+:     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+:     +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
++- LogicalProject(a3=[$0], b3=[$1], a1=[$2], b1=[$3])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T3]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashAggregate(isMerge=[true], groupBy=[a1, b1, a2, b2], select=[a1, b1, a2, b2])
++- Exchange(distribution=[hash[a1, b1, a2, b2]])
+   +- LocalHashAggregate(groupBy=[a1, b1, a2, b2], select=[a1, b1, a2, b2])
+      +- Union(all=[true], union=[a1, b1, a2, b2])
+         :- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+         :  :- Exchange(distribution=[hash[a1]])
+         :  :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+         :  +- Exchange(distribution=[hash[a2]])
+         :     +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+         +- SortMergeJoin(joinType=[InnerJoin], where=[=(a3, a1)], select=[a3, b3, a1, b1])
+            :- Exchange(distribution=[hash[a3]])
+            :  +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+            +- Exchange(distribution=[hash[a1]])
+               +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS NULL(a1), IS NULL(a2), =(a1, a2))], select=[a1, b1], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T2) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS NULL(a1), IS NULL(a2), =(a1, a2))], select=[a1, b1], build=[right])
+:- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[broadcast])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithDisabledOperator">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndLocalPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 inner join T2 on T1.a1 = T2.a2 and T1.a1 < 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($0, $2), <($0, 1))], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 inner join T2 on T1.b1 = T2.b2 and T1.a1 < 1 and T1.a1 < T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[AND(=($1, $3), <($0, 1), <($0, $2))], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[AND(=(b1, b2), <(a1, a2))], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[b1]])
+:  +- Calc(select=[a1, b1], where=[<(a1, 1)])
+:     +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1])
++- Exchange(distribution=[hash[b2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1, T2 where T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[=($0, $2)])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPredAndFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1, T2 where T1.a1 = T2.a2 and T1.a1 > 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalFilter(condition=[AND(=($0, $2), >($0, 1))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- 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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[FullOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T2) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[FullOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T2) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[LeftOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndLeftSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1, T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1, T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndRightSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T2, T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T2, T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide1">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1, T2]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0, 0] options:[T1, T2]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, a2, b2, a3, b3], build=[left])
+:- Exchange(distribution=[hash[b1]])
+:  +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide2">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1, T2]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0, 0] options:[T1, T2]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3], build=[left])
+:- Exchange(distribution=[hash[b2]])
+:  +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0, 0] options:[T1, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3])
+:- Exchange(distribution=[hash[b2]])
+:  +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T2, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0, 0] options:[T2, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(b1, b3)], select=[a1, b1, a2, b2, a3, b3])
+:- Exchange(distribution=[hash[b1]])
+:  +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5])
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))])
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T2, T3]]]])
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0, 0] options:[T2, T3]]]])
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(b2, b3)], select=[a1, b1, a2, b2, a3, b3])
+:- Exchange(distribution=[hash[b2]])
+:  +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
++- Exchange(distribution=[hash[b3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 inner join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/T1.* from T1 join V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutJoinPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1, T2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[RightOuterJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], select=[a1, b1], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T2) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], select=[a1, b1], build=[right])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2], hints=[[[ALIAS options:[T2]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName1">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(V22)*/V22.* from V22 join T1 on V22.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[V22]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[V2]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V22]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10])
+   :- 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])
+   :        +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2])
+   :           :- Exchange(distribution=[hash[a1]])
+   :           :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :           +- Exchange(distribution=[hash[a2]])
+   :              +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- Exchange(distribution=[hash[a1]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName2">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(V2)*/V2.* from V2 join T1 on V2.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[V2]]]])
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[V22]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]])
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]][ALIAS inheritPath:[0, 1, 0] options:[V2]]]])
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1, b1])
++- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10])
+   :- 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])
+   :        +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2])
+   :           :- Exchange(distribution=[hash[a1]])
+   :           :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+   :           +- Exchange(distribution=[hash[a2]])
+   :              +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+   +- Exchange(distribution=[hash[a1]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHints">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1), BROADCAST(T1),SHUFFLE_HASH(T1),NEST_LOOP(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]][BROADCAST inheritPath:[0] options:[T1]][SHUFFLE_HASH inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithTableAlias">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(V2)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[V2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]])
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+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])
+      +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[right])
+         :- Exchange(distribution=[hash[a1]])
+         :  +- TableSourceScan(table=[[default_catalog, default_database, T1, project=[a1], metadata=[]]], fields=[a1])
+         +- Exchange(distribution=[hash[a2]])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithView">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(V4) */* from T1 join V4 on T1.a1 = V4.a4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a4=[$2], b4=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[V4]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalProject(a4=[$0], b4=[$1], hints=[[[ALIAS options:[V4]]]])
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a3)], select=[a1, b1, a3, b3])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a3]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T3]], fields=[a3, b3])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHintsWithTheFirstOneIsInvalid">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1), NEST_LOOP(T1) */* from T1 join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], select=[a1, b1, a2, b2], build=[left])
+:- Exchange(distribution=[broadcast])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3])
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T2]]]])
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2, b2])
+:- Exchange(distribution=[hash[a1]])
+:  +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1])
++- Exchange(distribution=[hash[a2]])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2])
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml
new file mode 100644
index 00000000000..6408c6f0b54
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml
@@ -0,0 +1,702 @@
+<?xml version="1.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.
+-->
+<Root>
+  <TestCase name="testJoinHintInMultiLevelView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V3)*/V3.* from V3 join T1 on V3.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :     +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V2)*/T3.* from T3 join V2 on T3.a3 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a3=[$0], b3=[$1]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3, BIGINT a1, VARCHAR(2147483647) b1)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithCalc">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */a1 + 1, a1 * 10 from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(EXPR$0=[+($0, 1)], EXPR$1=[*($0, 10)]), rowType=[RecordType(BIGINT EXPR$0, BIGINT EXPR$1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithDuplicatedArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2 where T1.a1 > 5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalFilter(condition=[>($0, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndMultiArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2), BROADCAST(T2, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndSingleArg">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1), BROADCAST(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithUnion">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2 union select /*+ BROADCAST(T3) */* from T3 join T1 on T3.a3 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalUnion(all=[false]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+:- LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+:  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+:     :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+:     +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalProject(a3=[$0], b3=[$1], a1=[$2], b1=[$3]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3, BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3, BIGINT a1, VARCHAR(2147483647) b1)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithDisabledOperator">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndLocalPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 inner join T2 on T1.a1 = T2.a2 and T1.a1 < 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[AND(=($0, $2), <($0, 1))], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 inner join T2 on T1.b1 = T2.b2 and T1.a1 < 1 and T1.a1 < T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[AND(=($1, $3), <($0, 1), <($0, $2))], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1, T2 where T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalFilter(condition=[=($0, $2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPredAndFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1, T2 where T1.a1 = T2.a2 and T1.a1 > 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalFilter(condition=[AND(=($0, $2), >($0, 1))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndLeftSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndRightSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2, T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide1">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide2">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 inner join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutJoinPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1, T2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName1">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V22)*/V22.* from V22 join T1 on V22.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :     +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName2">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V2)*/V2.* from V2 join T1 on V2.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :     +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHints">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1), SHUFFLE_HASH(T1),SHUFFLE_MERGE(T1),NEST_LOOP(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][SHUFFLE_HASH options:[LEFT]][SHUFFLE_MERGE options:[LEFT]][NEST_LOOP options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithTableAlias">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V2)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V4) */* from T1 join V4 on T1.a1 = V4.a4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a4=[$2], b4=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a4, VARCHAR(2147483647) b4)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a4, VARCHAR(2147483647) b4)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalProject(a4=[$0], b4=[$1]), rowType=[RecordType(BIGINT a4, VARCHAR(2147483647) b4)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHintsWithTheFirstOneIsInvalid">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1), NEST_LOOP(T1) */* from T1 join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][NEST_LOOP options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+</Root>
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.xml
new file mode 100644
index 00000000000..e75c03efacc
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.xml
@@ -0,0 +1,702 @@
+<?xml version="1.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.
+-->
+<Root>
+  <TestCase name="testJoinHintInMultiLevelView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V3)*/V3.* from V3 join T1 on V3.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V3]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V2)*/T3.* from T3 join V2 on T3.a3 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a3=[$0], b3=[$1]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3, BIGINT a1, VARCHAR(2147483647) b1)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsOnSameViewWithReusingView">
+    <Resource name="ast">
+      <![CDATA[
+LogicalSink(table=[default_catalog.default_database.S1], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+
+LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+            +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+               :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+               +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithCalc">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */a1 + 1, a1 * 10 from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(EXPR$0=[+($0, 1)], EXPR$1=[*($0, 10)]), rowType=[RecordType(BIGINT EXPR$0, BIGINT EXPR$1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithDuplicatedArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2 where T1.a1 > 5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalFilter(condition=[>($0, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndMultiArgs">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2), BROADCAST(T2, T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithMultiSameJoinHintsAndSingleArg">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1), BROADCAST(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintsWithUnion">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2 union select /*+ BROADCAST(T3) */* from T3 join T1 on T3.a3 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalUnion(all=[false]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+:- LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+:  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+:     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+:     +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalProject(a3=[$0], b3=[$1], a1=[$2], b1=[$3]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3, BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3, BIGINT a1, VARCHAR(2147483647) b1)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithAntiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 where a1 not in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalFilter(condition=[NOT(IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+}))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithDisabledOperator">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndLocalPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 inner join T2 on T1.a1 = T2.a2 and T1.a1 < 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[AND(=($0, $2), <($0, 1))], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiAndNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 inner join T2 on T1.b1 = T2.b2 and T1.a1 < 1 and T1.a1 < T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[AND(=($1, $3), <($0, 1), <($0, $2))], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1, T2 where T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalFilter(condition=[=($0, $2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithEquiPredAndFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1, T2 where T1.a1 = T2.a2 and T1.a1 > 1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalFilter(condition=[AND(=($0, $2), >($0, 1))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithFullJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 full join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[full], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithLeftJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 left join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[left], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndLeftSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiArgsAndRightSideFirst">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2, T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide1">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstSideAsBuildSide2">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($1, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3], a3=[$4], b3=[$5]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
++- LogicalFilter(condition=[AND(=($0, $2), =($3, $5))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+   +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2, BIGINT a3, VARCHAR(2147483647) b3)]
+      :- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+      :  :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      :  +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]], hints=[[[ALIAS inheritPath:[] options:[T3]]]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithNonEquiPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 inner join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithoutJoinPred">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1, T2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithRightJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 right join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[right], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSemiJoinAndRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 where a1 in (select a2 from T2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalFilter(condition=[IN($0, {
+LogicalProject(a2=[$0])
+  LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+})]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName1">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V22)*/V22.* from V22 join T1 on V22.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]][ALIAS inheritPath:[0, 1, 0] options:[V22]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithSubStringViewName2">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V2)*/V2.* from V2 join T1 on V2.a1 = T1.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :  +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :     :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :     +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V22]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :        +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V22]][ALIAS inheritPath:[0, 1, 0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :           :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   :           +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHints">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1), SHUFFLE_HASH(T1),SHUFFLE_MERGE(T1),NEST_LOOP(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][SHUFFLE_HASH options:[LEFT]][SHUFFLE_MERGE options:[LEFT]][NEST_LOOP options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithTableAlias">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V2)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+         :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testJoinHintWithView">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(V4) */* from T1 join V4 on T1.a1 = V4.a4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a4=[$2], b4=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a4, VARCHAR(2147483647) b4)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a4, VARCHAR(2147483647) b4)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalProject(a4=[$0], b4=[$1], hints=[[[ALIAS options:[V4]]]]), rowType=[RecordType(BIGINT a4, VARCHAR(2147483647) b4)]
+      +- LogicalTableScan(table=[[default_catalog, default_database, T3]]), rowType=[RecordType(BIGINT a3, VARCHAR(2147483647) b3)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testMultiJoinHintsWithTheFirstOneIsInvalid">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1), NEST_LOOP(T1) */* from T1 join T2 on T1.a1 > T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[>($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][NEST_LOOP options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithLeftSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase name="testSimpleJoinHintWithRightSideAsBuildSide">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T2) */* from T1 join T2 on T1.a1 = T2.a2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
++- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)]
+   :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+</Root>
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 42099c58e14..dc2500d0242 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
@@ -107,12 +107,4 @@ class ShuffledHashJoinTest extends JoinTestBase {
     thrown.expectMessage("Cannot generate a valid execution plan for the given query")
     super.testCrossJoin()
   }
-
-  @Test
-  override def testSelfJoin(): Unit = {
-    // TODO use shuffle hash join if isBroadcast is true and isBroadcastHashJoinEnabled is false ?
-    thrown.expect(classOf[TableException])
-    thrown.expectMessage("Cannot generate a valid execution plan for the given query")
-    super.testSelfJoin()
-  }
 }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryTestBase.scala
index cfe212e5b23..9204b27a4a6 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryTestBase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryTestBase.scala
@@ -18,6 +18,7 @@
 package org.apache.flink.table.planner.plan.rules.logical.subquery
 
 import org.apache.flink.table.planner.calcite.CalciteConfig
+import org.apache.flink.table.planner.hint.FlinkHintStrategies
 import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram
 import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableConfigUtils, TableTestBase}
 
@@ -35,7 +36,8 @@ class SubQueryTestBase extends TableTestBase {
       .config()
       .withTrimUnusedFields(false)
       .withExpand(false)
-      .withInSubQueryThreshold(3))
+      .withInSubQueryThreshold(3)
+      .withHintStrategyTable(FlinkHintStrategies.createHintStrategyTable()))
 
   util.tableEnv.getConfig.setPlannerConfig(builder.build())
 }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
index 1b37f8dcb86..fd2fd085519 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
@@ -300,7 +300,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
       query,
       Array.empty[ExplainDetail],
       withRowType = false,
-      Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC))
+      Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC),
+      withQueryBlockAlias = false)
   }
 
   /**
@@ -313,7 +314,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
       query,
       extraDetails.toArray,
       withRowType = false,
-      Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC))
+      Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC),
+      withQueryBlockAlias = false)
   }
 
   /**
@@ -406,7 +408,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
       extraDetails.toArray,
       withRowType = false,
       Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC),
-      () => Unit)
+      () => Unit,
+      withQueryBlockAlias = false)
   }
 
   /** Verify the AST (abstract syntax tree). */
@@ -416,7 +419,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
       Array.empty[ExplainDetail],
       withRowType = false,
       Array(PlanKind.AST),
-      () => Unit)
+      () => Unit,
+      withQueryBlockAlias = false)
   }
 
   /** Verify the AST (abstract syntax tree). The plans will contain the extra [[ExplainDetail]]s. */
@@ -426,7 +430,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
       extraDetails.toArray,
       withRowType = false,
       Array(PlanKind.AST),
-      () => Unit)
+      () => Unit,
+      withQueryBlockAlias = false)
   }
 
   /**
@@ -437,7 +442,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
       query,
       Array.empty[ExplainDetail],
... 171 lines suppressed ...


[flink] 01/04: [FLINK-28678][table-planner] Copy some nodes from calcite, and need revert when upgrade calcite-1.31

Posted by go...@apache.org.
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

commit d17dcbd43b287c58efc726f8a0a9b78c5ddf7f60
Author: xuyang <xy...@163.com>
AuthorDate: Tue Jul 26 14:08:26 2022 +0800

    [FLINK-28678][table-planner] Copy some nodes from calcite, and need revert when upgrade calcite-1.31
    
    Since currently the feat [CALCITE-5107] is not introduced to the calcite version flink used, so just temporarily copy the files and cherry-pick this feat directly. When the calcite upgrades to 1.31, these classes should be deleted.
    
    This pr is just to copy some nodes from calcite, but doesn't do any modification on them.
    
    This closes #20359
---
 .../org/apache/calcite/rel/core/Correlate.java     | 234 +++++++++++
 .../java/org/apache/calcite/rel/core/Filter.java   | 169 ++++++++
 .../org/apache/calcite/rel/core/Intersect.java     |  57 +++
 .../java/org/apache/calcite/rel/core/Minus.java    |  54 +++
 .../java/org/apache/calcite/rel/core/SetOp.java    | 133 +++++++
 .../java/org/apache/calcite/rel/core/Sort.java     | 197 ++++++++++
 .../java/org/apache/calcite/rel/core/Union.java    |  65 +++
 .../java/org/apache/calcite/rel/core/Values.java   | 210 ++++++++++
 .../java/org/apache/calcite/rel/core/Window.java   | 435 +++++++++++++++++++++
 .../apache/calcite/rel/hint/HintPredicates.java    |  86 ++++
 .../calcite/rel/hint/NodeTypeHintPredicate.java    |  80 ++++
 .../calcite/rel/logical/LogicalCorrelate.java      | 116 ++++++
 .../apache/calcite/rel/logical/LogicalFilter.java  | 145 +++++++
 .../calcite/rel/logical/LogicalIntersect.java      |  76 ++++
 .../apache/calcite/rel/logical/LogicalMinus.java   |  74 ++++
 .../apache/calcite/rel/logical/LogicalSort.java    |  82 ++++
 .../apache/calcite/rel/logical/LogicalUnion.java   |  73 ++++
 .../apache/calcite/rel/logical/LogicalValues.java  | 134 +++++++
 .../apache/calcite/rel/logical/LogicalWindow.java  | 367 +++++++++++++++++
 19 files changed, 2787 insertions(+)

diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Correlate.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Correlate.java
new file mode 100644
index 00000000000..2d5e6fe554f
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Correlate.java
@@ -0,0 +1,234 @@
+/*
+ * 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.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * A relational operator that performs nested-loop joins.
+ *
+ * <p>It behaves like a kind of {@link org.apache.calcite.rel.core.Join}, but works by setting
+ * variables in its environment and restarting its right-hand input.
+ *
+ * <p>Correlate is not a join since: typical rules should not match Correlate.
+ *
+ * <p>A Correlate is used to represent a correlated query. One implementation strategy is to
+ * de-correlate the expression.
+ *
+ * <table>
+ *   <caption>Mapping of physical operations to logical ones</caption>
+ *   <tr><th>Physical operation</th><th>Logical operation</th></tr>
+ *   <tr><td>NestedLoops</td><td>Correlate(A, B, regular)</td></tr>
+ *   <tr><td>NestedLoopsOuter</td><td>Correlate(A, B, outer)</td></tr>
+ *   <tr><td>NestedLoopsSemi</td><td>Correlate(A, B, semi)</td></tr>
+ *   <tr><td>NestedLoopsAnti</td><td>Correlate(A, B, anti)</td></tr>
+ *   <tr><td>HashJoin</td><td>EquiJoin(A, B)</td></tr>
+ *   <tr><td>HashJoinOuter</td><td>EquiJoin(A, B, outer)</td></tr>
+ *   <tr><td>HashJoinSemi</td><td>SemiJoin(A, B, semi)</td></tr>
+ *   <tr><td>HashJoinAnti</td><td>SemiJoin(A, B, anti)</td></tr>
+ * </table>
+ *
+ * @see CorrelationId
+ */
+public abstract class Correlate extends BiRel {
+    // ~ Instance fields --------------------------------------------------------
+
+    protected final CorrelationId correlationId;
+    protected final ImmutableBitSet requiredColumns;
+    protected final JoinRelType joinType;
+
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a Correlate.
+     *
+     * @param cluster Cluster this relational expression belongs to
+     * @param left Left input relational expression
+     * @param right Right input relational expression
+     * @param correlationId Variable name for the row of left input
+     * @param requiredColumns Set of columns that are used by correlation
+     * @param joinType Join type
+     */
+    protected Correlate(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode left,
+            RelNode right,
+            CorrelationId correlationId,
+            ImmutableBitSet requiredColumns,
+            JoinRelType joinType) {
+        super(cluster, traitSet, left, right);
+        assert !joinType.generatesNullsOnLeft() : "Correlate has invalid join type " + joinType;
+        this.joinType = Objects.requireNonNull(joinType);
+        this.correlationId = Objects.requireNonNull(correlationId);
+        this.requiredColumns = Objects.requireNonNull(requiredColumns);
+    }
+
+    /**
+     * Creates a Correlate by parsing serialized output.
+     *
+     * @param input Input representation
+     */
+    public Correlate(RelInput input) {
+        this(
+                input.getCluster(),
+                input.getTraitSet(),
+                input.getInputs().get(0),
+                input.getInputs().get(1),
+                new CorrelationId((Integer) input.get("correlation")),
+                input.getBitSet("requiredColumns"),
+                input.getEnum("joinType", JoinRelType.class));
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    @Override
+    public boolean isValid(Litmus litmus, RelNode.Context context) {
+        return super.isValid(litmus, context)
+                && RelOptUtil.notContainsCorrelation(left, correlationId, litmus);
+    }
+
+    @Override
+    public Correlate copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        assert inputs.size() == 2;
+        return copy(
+                traitSet, inputs.get(0), inputs.get(1), correlationId, requiredColumns, joinType);
+    }
+
+    public abstract Correlate copy(
+            RelTraitSet traitSet,
+            RelNode left,
+            RelNode right,
+            CorrelationId correlationId,
+            ImmutableBitSet requiredColumns,
+            JoinRelType joinType);
+
+    public JoinRelType getJoinType() {
+        return joinType;
+    }
+
+    @Override
+    protected RelDataType deriveRowType() {
+        switch (joinType) {
+            case LEFT:
+            case INNER:
+                return SqlValidatorUtil.deriveJoinRowType(
+                        left.getRowType(),
+                        right.getRowType(),
+                        joinType,
+                        getCluster().getTypeFactory(),
+                        null,
+                        com.google.common.collect.ImmutableList.of());
+            case ANTI:
+            case SEMI:
+                return left.getRowType();
+            default:
+                throw new IllegalStateException("Unknown join type " + joinType);
+        }
+    }
+
+    @Override
+    public RelWriter explainTerms(RelWriter pw) {
+        return super.explainTerms(pw)
+                .item("correlation", correlationId)
+                .item("joinType", joinType.lowerName)
+                .item("requiredColumns", requiredColumns);
+    }
+
+    /**
+     * Returns the correlating expressions.
+     *
+     * @return correlating expressions
+     */
+    public CorrelationId getCorrelationId() {
+        return correlationId;
+    }
+
+    @Override
+    public String getCorrelVariable() {
+        return correlationId.getName();
+    }
+
+    /**
+     * Returns the required columns in left relation required for the correlation in the right.
+     *
+     * @return columns in left relation required for the correlation in the right
+     */
+    public ImmutableBitSet getRequiredColumns() {
+        return requiredColumns;
+    }
+
+    @Override
+    public Set<CorrelationId> getVariablesSet() {
+        return com.google.common.collect.ImmutableSet.of(correlationId);
+    }
+
+    @Override
+    public double estimateRowCount(RelMetadataQuery mq) {
+        double leftRowCount = mq.getRowCount(left);
+        switch (joinType) {
+            case SEMI:
+            case ANTI:
+                return leftRowCount;
+            default:
+                return leftRowCount * mq.getRowCount(right);
+        }
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        double rowCount = mq.getRowCount(this);
+
+        final double rightRowCount = right.estimateRowCount(mq);
+        final double leftRowCount = left.estimateRowCount(mq);
+        if (Double.isInfinite(leftRowCount) || Double.isInfinite(rightRowCount)) {
+            return planner.getCostFactory().makeInfiniteCost();
+        }
+
+        Double restartCount = mq.getRowCount(getLeft());
+        // RelMetadataQuery.getCumulativeCost(getRight()); does not work for
+        // RelSubset, so we ask planner to cost-estimate right relation
+        RelOptCost rightCost = planner.getCost(getRight(), mq);
+        RelOptCost rescanCost = rightCost.multiplyBy(Math.max(1.0, restartCount - 1));
+
+        return planner.getCostFactory()
+                .makeCost(
+                        rowCount /* generate results */ + leftRowCount /* scan left results */,
+                        0,
+                        0)
+                .plus(rescanCost);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Filter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Filter.java
new file mode 100644
index 00000000000..2fc9f0c1240
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Filter.java
@@ -0,0 +1,169 @@
+/*
+ * 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.calcite.rel.core;
+
+import org.apache.calcite.config.CalciteSystemProperty;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexChecker;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.Litmus;
+
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Relational expression that iterates over its input and returns elements for which <code>condition
+ * </code> evaluates to <code>true</code>.
+ *
+ * <p>If the condition allows nulls, then a null value is treated the same as false.
+ *
+ * @see org.apache.calcite.rel.logical.LogicalFilter
+ */
+public abstract class Filter extends SingleRel {
+    // ~ Instance fields --------------------------------------------------------
+
+    protected final RexNode condition;
+
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a filter.
+     *
+     * @param cluster Cluster that this relational expression belongs to
+     * @param traits the traits of this rel
+     * @param child input relational expression
+     * @param condition boolean expression which determines whether a row is allowed to pass
+     */
+    protected Filter(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+        super(cluster, traits, child);
+        assert condition != null;
+        assert RexUtil.isFlat(condition) : condition;
+        this.condition = condition;
+        // Too expensive for everyday use:
+        assert !CalciteSystemProperty.DEBUG.value() || isValid(Litmus.THROW, null);
+    }
+
+    /** Creates a Filter by parsing serialized output. */
+    protected Filter(RelInput input) {
+        this(
+                input.getCluster(),
+                input.getTraitSet(),
+                input.getInput(),
+                input.getExpression("condition"));
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    @Override
+    public final RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return copy(traitSet, sole(inputs), getCondition());
+    }
+
+    public abstract Filter copy(RelTraitSet traitSet, RelNode input, RexNode condition);
+
+    public RelNode accept(RexShuttle shuttle) {
+        RexNode condition = shuttle.apply(this.condition);
+        if (this.condition == condition) {
+            return this;
+        }
+        return copy(traitSet, getInput(), condition);
+    }
+
+    public RexNode getCondition() {
+        return condition;
+    }
+
+    /** Returns whether this Filter contains any windowed-aggregate functions. */
+    public final boolean containsOver() {
+        return RexOver.containsOver(condition);
+    }
+
+    @Override
+    public boolean isValid(Litmus litmus, Context context) {
+        if (RexUtil.isNullabilityCast(getCluster().getTypeFactory(), condition)) {
+            return litmus.fail("Cast for just nullability not allowed");
+        }
+        final RexChecker checker = new RexChecker(getInput().getRowType(), context, litmus);
+        condition.accept(checker);
+        if (checker.getFailureCount() > 0) {
+            return litmus.fail(null);
+        }
+        return litmus.succeed();
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        double dRows = mq.getRowCount(this);
+        double dCpu = mq.getRowCount(getInput());
+        double dIo = 0;
+        return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
+    }
+
+    @Override
+    public double estimateRowCount(RelMetadataQuery mq) {
+        return RelMdUtil.estimateFilteredRows(getInput(), condition, mq);
+    }
+
+    @Deprecated // to be removed before 2.0
+    public static double estimateFilteredRows(RelNode child, RexProgram program) {
+        final RelMetadataQuery mq = child.getCluster().getMetadataQuery();
+        return RelMdUtil.estimateFilteredRows(child, program, mq);
+    }
+
+    @Deprecated // to be removed before 2.0
+    public static double estimateFilteredRows(RelNode child, RexNode condition) {
+        final RelMetadataQuery mq = child.getCluster().getMetadataQuery();
+        return RelMdUtil.estimateFilteredRows(child, condition, mq);
+    }
+
+    public RelWriter explainTerms(RelWriter pw) {
+        return super.explainTerms(pw).item("condition", condition);
+    }
+
+    protected boolean deepEquals0(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+        if (obj == null || getClass() != obj.getClass()) {
+            return false;
+        }
+        Filter o = (Filter) obj;
+        return traitSet.equals(o.traitSet)
+                && input.deepEquals(o.input)
+                && condition.equals(o.condition)
+                && getRowType().equalsSansFieldNames(o.getRowType());
+    }
+
+    protected int deepHashCode0() {
+        return Objects.hash(traitSet, input.deepHashCode(), condition);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Intersect.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Intersect.java
new file mode 100644
index 00000000000..c923692275b
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Intersect.java
@@ -0,0 +1,57 @@
+/*
+ * 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.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.sql.SqlKind;
+
+import java.util.List;
+
+/**
+ * Relational expression that returns the intersection of the rows of its inputs.
+ *
+ * <p>If "all" is true, performs then multiset intersection; otherwise, performs set set
+ * intersection (implying no duplicates in the results).
+ */
+public abstract class Intersect extends SetOp {
+    /** Creates an Intersect. */
+    public Intersect(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
+        super(cluster, traits, inputs, SqlKind.INTERSECT, all);
+    }
+
+    /** Creates an Intersect by parsing serialized output. */
+    protected Intersect(RelInput input) {
+        super(input);
+    }
+
+    @Override
+    public double estimateRowCount(RelMetadataQuery mq) {
+        // REVIEW jvs 30-May-2005:  I just pulled this out of a hat.
+        double dRows = Double.MAX_VALUE;
+        for (RelNode input : inputs) {
+            dRows = Math.min(dRows, mq.getRowCount(input));
+        }
+        dRows *= 0.25;
+        return dRows;
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Minus.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Minus.java
new file mode 100644
index 00000000000..035581e4150
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Minus.java
@@ -0,0 +1,54 @@
+/*
+ * 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.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.sql.SqlKind;
+
+import java.util.List;
+
+/**
+ * Relational expression that returns the rows of its first input minus any matching rows from its
+ * other inputs.
+ *
+ * <p>Corresponds to the SQL {@code EXCEPT} operator.
+ *
+ * <p>If "all" is true, then multiset subtraction is performed; otherwise, set subtraction is
+ * performed (implying no duplicates in the results).
+ */
+public abstract class Minus extends SetOp {
+    public Minus(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
+        super(cluster, traits, inputs, SqlKind.EXCEPT, all);
+    }
+
+    /** Creates a Minus by parsing serialized output. */
+    protected Minus(RelInput input) {
+        super(input);
+    }
+
+    @Override
+    public double estimateRowCount(RelMetadataQuery mq) {
+        return RelMdUtil.getMinusRowCount(mq, this);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/SetOp.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/SetOp.java
new file mode 100644
index 00000000000..2e323031102
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/SetOp.java
@@ -0,0 +1,133 @@
+/*
+ * 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.calcite.rel.core;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.Util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * <code>SetOp</code> is an abstract base for relational set operators such as UNION, MINUS (aka
+ * EXCEPT), and INTERSECT.
+ */
+public abstract class SetOp extends AbstractRelNode {
+    // ~ Instance fields --------------------------------------------------------
+
+    protected com.google.common.collect.ImmutableList<RelNode> inputs;
+    public final SqlKind kind;
+    public final boolean all;
+
+    // ~ Constructors -----------------------------------------------------------
+
+    /** Creates a SetOp. */
+    protected SetOp(
+            RelOptCluster cluster,
+            RelTraitSet traits,
+            List<RelNode> inputs,
+            SqlKind kind,
+            boolean all) {
+        super(cluster, traits);
+        com.google.common.base.Preconditions.checkArgument(
+                kind == SqlKind.UNION || kind == SqlKind.INTERSECT || kind == SqlKind.EXCEPT);
+        this.kind = kind;
+        this.inputs = com.google.common.collect.ImmutableList.copyOf(inputs);
+        this.all = all;
+    }
+
+    /** Creates a SetOp by parsing serialized output. */
+    protected SetOp(RelInput input) {
+        this(
+                input.getCluster(),
+                input.getTraitSet(),
+                input.getInputs(),
+                SqlKind.UNION,
+                input.getBoolean("all", false));
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    public abstract SetOp copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all);
+
+    @Override
+    public SetOp copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return copy(traitSet, inputs, all);
+    }
+
+    @Override
+    public void replaceInput(int ordinalInParent, RelNode p) {
+        final List<RelNode> newInputs = new ArrayList<>(inputs);
+        newInputs.set(ordinalInParent, p);
+        inputs = com.google.common.collect.ImmutableList.copyOf(newInputs);
+        recomputeDigest();
+    }
+
+    @Override
+    public List<RelNode> getInputs() {
+        return inputs;
+    }
+
+    @Override
+    public RelWriter explainTerms(RelWriter pw) {
+        super.explainTerms(pw);
+        for (Ord<RelNode> ord : Ord.zip(inputs)) {
+            pw.input("input#" + ord.i, ord.e);
+        }
+        return pw.item("all", all);
+    }
+
+    @Override
+    protected RelDataType deriveRowType() {
+        final List<RelDataType> inputRowTypes = Util.transform(inputs, RelNode::getRowType);
+        final RelDataType rowType = getCluster().getTypeFactory().leastRestrictive(inputRowTypes);
+        if (rowType == null) {
+            throw new IllegalArgumentException(
+                    "Cannot compute compatible row type "
+                            + "for arguments to set op: "
+                            + Util.sepList(inputRowTypes, ", "));
+        }
+        return rowType;
+    }
+
+    /**
+     * Returns whether all the inputs of this set operator have the same row type as its output row.
+     *
+     * @param compareNames Whether column names are important in the homogeneity comparison
+     * @return Whether all the inputs of this set operator have the same row type as its output row
+     */
+    public boolean isHomogeneous(boolean compareNames) {
+        RelDataType unionType = getRowType();
+        for (RelNode input : getInputs()) {
+            if (!RelOptUtil.areRowTypesEqual(input.getRowType(), unionType, compareNames)) {
+                return false;
+            }
+        }
+        return true;
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Sort.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Sort.java
new file mode 100644
index 00000000000..8ebab6e03be
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Sort.java
@@ -0,0 +1,197 @@
+/*
+ * 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.calcite.rel.core;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.util.Util;
+
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * Relational expression that imposes a particular sort order on its input without otherwise
+ * changing its content.
+ */
+public abstract class Sort extends SingleRel {
+    // ~ Instance fields --------------------------------------------------------
+
+    public final RelCollation collation;
+    public final RexNode offset;
+    public final RexNode fetch;
+
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a Sort.
+     *
+     * @param cluster Cluster this relational expression belongs to
+     * @param traits Traits
+     * @param child input relational expression
+     * @param collation array of sort specifications
+     */
+    public Sort(RelOptCluster cluster, RelTraitSet traits, RelNode child, RelCollation collation) {
+        this(cluster, traits, child, collation, null, null);
+    }
+
+    /**
+     * Creates a Sort.
+     *
+     * @param cluster Cluster this relational expression belongs to
+     * @param traits Traits
+     * @param child input relational expression
+     * @param collation array of sort specifications
+     * @param offset Expression for number of rows to discard before returning first row
+     * @param fetch Expression for number of rows to fetch
+     */
+    public Sort(
+            RelOptCluster cluster,
+            RelTraitSet traits,
+            RelNode child,
+            RelCollation collation,
+            RexNode offset,
+            RexNode fetch) {
+        super(cluster, traits, child);
+        this.collation = collation;
+        this.offset = offset;
+        this.fetch = fetch;
+
+        assert traits.containsIfApplicable(collation)
+                : "traits=" + traits + ", collation=" + collation;
+        assert !(fetch == null && offset == null && collation.getFieldCollations().isEmpty())
+                : "trivial sort";
+    }
+
+    /** Creates a Sort by parsing serialized output. */
+    public Sort(RelInput input) {
+        this(
+                input.getCluster(),
+                input.getTraitSet().plus(input.getCollation()),
+                input.getInput(),
+                RelCollationTraitDef.INSTANCE.canonize(input.getCollation()),
+                input.getExpression("offset"),
+                input.getExpression("fetch"));
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    @Override
+    public final Sort copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return copy(traitSet, sole(inputs), collation, offset, fetch);
+    }
+
+    public final Sort copy(RelTraitSet traitSet, RelNode newInput, RelCollation newCollation) {
+        return copy(traitSet, newInput, newCollation, offset, fetch);
+    }
+
+    public abstract Sort copy(
+            RelTraitSet traitSet,
+            RelNode newInput,
+            RelCollation newCollation,
+            RexNode offset,
+            RexNode fetch);
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        // Higher cost if rows are wider discourages pushing a project through a
+        // sort.
+        final double rowCount = mq.getRowCount(this);
+        final double bytesPerRow = getRowType().getFieldCount() * 4;
+        final double cpu = Util.nLogN(rowCount) * bytesPerRow;
+        return planner.getCostFactory().makeCost(rowCount, cpu, 0);
+    }
+
+    public RelNode accept(RexShuttle shuttle) {
+        RexNode offset = shuttle.apply(this.offset);
+        RexNode fetch = shuttle.apply(this.fetch);
+        List<RexNode> originalSortExps = getSortExps();
+        List<RexNode> sortExps = shuttle.apply(originalSortExps);
+        assert sortExps == originalSortExps
+                : "Sort node does not support modification of input field expressions."
+                        + " Old expressions: "
+                        + originalSortExps
+                        + ", new ones: "
+                        + sortExps;
+        if (offset == this.offset && fetch == this.fetch) {
+            return this;
+        }
+        return copy(traitSet, getInput(), collation, offset, fetch);
+    }
+
+    @Override
+    public boolean isEnforcer() {
+        return offset == null && fetch == null && collation.getFieldCollations().size() > 0;
+    }
+
+    /**
+     * Returns the array of {@link RelFieldCollation}s asked for by the sort specification, from
+     * most significant to least significant.
+     *
+     * <p>See also {@link RelMetadataQuery#collations(RelNode)}, which lists all known collations.
+     * For example, <code>ORDER BY time_id</code> might also be sorted by <code>the_year, the_month
+     * </code> because of a known monotonicity constraint among the columns. {@code getCollation}
+     * would return <code>[time_id]</code> and {@code collations} would return <code>
+     * [ [time_id], [the_year, the_month] ]</code>.
+     */
+    public RelCollation getCollation() {
+        return collation;
+    }
+
+    /** Returns the sort expressions. */
+    public List<RexNode> getSortExps() {
+        //noinspection StaticPseudoFunctionalStyleMethod
+        return Util.transform(
+                collation.getFieldCollations(),
+                field ->
+                        getCluster()
+                                .getRexBuilder()
+                                .makeInputRef(
+                                        input, Objects.requireNonNull(field).getFieldIndex()));
+    }
+
+    public RelWriter explainTerms(RelWriter pw) {
+        super.explainTerms(pw);
+        if (pw.nest()) {
+            pw.item("collation", collation);
+        } else {
+            for (Ord<RexNode> ord : Ord.zip(getSortExps())) {
+                pw.item("sort" + ord.i, ord.e);
+            }
+            for (Ord<RelFieldCollation> ord : Ord.zip(collation.getFieldCollations())) {
+                pw.item("dir" + ord.i, ord.e.shortString());
+            }
+        }
+        pw.itemIf("offset", offset, offset != null);
+        pw.itemIf("fetch", fetch, fetch != null);
+        return pw;
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Union.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Union.java
new file mode 100644
index 00000000000..f79538b45e7
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Union.java
@@ -0,0 +1,65 @@
+/*
+ * 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.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.metadata.RelMdUtil;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.sql.SqlKind;
+
+import java.util.List;
+
+/**
+ * Relational expression that returns the union of the rows of its inputs, optionally eliminating
+ * duplicates.
+ *
+ * <p>Corresponds to SQL {@code UNION} and {@code UNION ALL}.
+ */
+public abstract class Union extends SetOp {
+    // ~ Constructors -----------------------------------------------------------
+
+    protected Union(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
+        super(cluster, traits, inputs, SqlKind.UNION, all);
+    }
+
+    /** Creates a Union by parsing serialized output. */
+    protected Union(RelInput input) {
+        super(input);
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    @Override
+    public double estimateRowCount(RelMetadataQuery mq) {
+        double dRows = RelMdUtil.getUnionAllRowCount(mq, this);
+        if (!all) {
+            dRows *= 0.5;
+        }
+        return dRows;
+    }
+
+    @Deprecated // to be removed before 2.0
+    public static double estimateRowCount(RelNode rel) {
+        final RelMetadataQuery mq = rel.getCluster().getMetadataQuery();
+        return RelMdUtil.getUnionAllRowCount(mq, (Union) rel);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Values.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Values.java
new file mode 100644
index 00000000000..7aa0d0cafb0
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Values.java
@@ -0,0 +1,210 @@
+/*
+ * 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.calcite.rel.core;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexDigestIncludeType;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.Pair;
+
+import java.util.List;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+/** Relational expression whose value is a sequence of zero or more literal row values. */
+public abstract class Values extends AbstractRelNode {
+
+    public static final Predicate<? super Values> IS_EMPTY_J = Values::isEmpty;
+
+    @SuppressWarnings("Guava")
+    @Deprecated // to be removed before 2.0
+    public static final com.google.common.base.Predicate<? super Values> IS_EMPTY = Values::isEmpty;
+
+    @SuppressWarnings("Guava")
+    @Deprecated // to be removed before 2.0
+    public static final com.google.common.base.Predicate<? super Values> IS_NOT_EMPTY =
+            Values::isNotEmpty;
+
+    // ~ Instance fields --------------------------------------------------------
+
+    public final com.google.common.collect.ImmutableList<
+                    com.google.common.collect.ImmutableList<RexLiteral>>
+            tuples;
+
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a new Values.
+     *
+     * <p>Note that tuples passed in become owned by this rel (without a deep copy), so caller must
+     * not modify them after this call, otherwise bad things will happen.
+     *
+     * @param cluster Cluster that this relational expression belongs to
+     * @param rowType Row type for tuples produced by this rel
+     * @param tuples 2-dimensional array of tuple values to be produced; outer list contains tuples;
+     *     each inner list is one tuple; all tuples must be of same length, conforming to rowType
+     */
+    protected Values(
+            RelOptCluster cluster,
+            RelDataType rowType,
+            com.google.common.collect.ImmutableList<
+                            com.google.common.collect.ImmutableList<RexLiteral>>
+                    tuples,
+            RelTraitSet traits) {
+        super(cluster, traits);
+        this.rowType = rowType;
+        this.tuples = tuples;
+        assert assertRowType();
+    }
+
+    /** Creates a Values by parsing serialized output. */
+    public Values(RelInput input) {
+        this(
+                input.getCluster(),
+                input.getRowType("type"),
+                input.getTuples("tuples"),
+                input.getTraitSet());
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    /**
+     * Predicate, to be used when defining an operand of a {@link RelOptRule}, that returns true if
+     * a Values contains zero tuples.
+     *
+     * <p>This is the conventional way to represent an empty relational expression. There are
+     * several rules that recognize empty relational expressions and prune away that section of the
+     * tree.
+     */
+    public static boolean isEmpty(Values values) {
+        return values.getTuples().isEmpty();
+    }
+
+    /**
+     * Predicate, to be used when defining an operand of a {@link RelOptRule}, that returns true if
+     * a Values contains one or more tuples.
+     *
+     * <p>This is the conventional way to represent an empty relational expression. There are
+     * several rules that recognize empty relational expressions and prune away that section of the
+     * tree.
+     */
+    public static boolean isNotEmpty(Values values) {
+        return !isEmpty(values);
+    }
+
+    public com.google.common.collect.ImmutableList<
+                    com.google.common.collect.ImmutableList<RexLiteral>>
+            getTuples(RelInput input) {
+        return input.getTuples("tuples");
+    }
+
+    /** Returns the rows of literals represented by this Values relational expression. */
+    public com.google.common.collect.ImmutableList<
+                    com.google.common.collect.ImmutableList<RexLiteral>>
+            getTuples() {
+        return tuples;
+    }
+
+    /** Returns true if all tuples match rowType; otherwise, assert on mismatch. */
+    private boolean assertRowType() {
+        for (List<RexLiteral> tuple : tuples) {
+            assert tuple.size() == rowType.getFieldCount();
+            for (Pair<RexLiteral, RelDataTypeField> pair :
+                    Pair.zip(tuple, rowType.getFieldList())) {
+                RexLiteral literal = pair.left;
+                RelDataType fieldType = pair.right.getType();
+
+                // TODO jvs 19-Feb-2006: strengthen this a bit.  For example,
+                // overflow, rounding, and padding/truncation must already have
+                // been dealt with.
+                if (!RexLiteral.isNullLiteral(literal)) {
+                    assert SqlTypeUtil.canAssignFrom(fieldType, literal.getType())
+                            : "to " + fieldType + " from " + literal;
+                }
+            }
+        }
+        return true;
+    }
+
+    @Override
+    protected RelDataType deriveRowType() {
+        return rowType;
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        double dRows = mq.getRowCount(this);
+
+        // Assume CPU is negligible since values are precomputed.
+        double dCpu = 1;
+        double dIo = 0;
+        return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
+    }
+
+    // implement RelNode
+    public double estimateRowCount(RelMetadataQuery mq) {
+        return tuples.size();
+    }
+
+    // implement RelNode
+    public RelWriter explainTerms(RelWriter pw) {
+        // A little adapter just to get the tuples to come out
+        // with curly brackets instead of square brackets.  Plus
+        // more whitespace for readability.
+        RelWriter relWriter =
+                super.explainTerms(pw)
+                        // For rel digest, include the row type since a rendered
+                        // literal may leave the type ambiguous (e.g. "null").
+                        .itemIf(
+                                "type",
+                                rowType,
+                                pw.getDetailLevel() == SqlExplainLevel.DIGEST_ATTRIBUTES)
+                        .itemIf("type", rowType.getFieldList(), pw.nest());
+        if (pw.nest()) {
+            pw.item("tuples", tuples);
+        } else {
+            pw.item(
+                    "tuples",
+                    tuples.stream()
+                            .map(
+                                    row ->
+                                            row.stream()
+                                                    .map(
+                                                            lit ->
+                                                                    lit.computeDigest(
+                                                                            RexDigestIncludeType
+                                                                                    .NO_TYPE))
+                                                    .collect(Collectors.joining(", ", "{ ", " }")))
+                            .collect(Collectors.joining(", ", "[", "]")));
+        }
+        return relWriter;
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Window.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Window.java
new file mode 100644
index 00000000000..26f4de75f9e
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Window.java
@@ -0,0 +1,435 @@
+/*
+ * 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.calcite.rel.core;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptCost;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollations;
+import org.apache.calcite.rel.RelFieldCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexChecker;
+import org.apache.calcite.rex.RexFieldCollation;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexSlot;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.sql.SqlAggFunction;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Util;
+
+import java.util.AbstractList;
+import java.util.List;
+import java.util.Objects;
+
+/**
+ * A relational expression representing a set of window aggregates.
+ *
+ * <p>A Window can handle several window aggregate functions, over several partitions, with pre- and
+ * post-expressions, and an optional post-filter. Each of the partitions is defined by a partition
+ * key (zero or more columns) and a range (logical or physical). The partitions expect the data to
+ * be sorted correctly on input to the relational expression.
+ *
+ * <p>Each {@link Group} has a set of {@link org.apache.calcite.rex.RexOver} objects.
+ *
+ * <p>Created by {@link org.apache.calcite.rel.rules.ProjectToWindowRule}.
+ */
+public abstract class Window extends SingleRel {
+    public final com.google.common.collect.ImmutableList<Group> groups;
+    public final com.google.common.collect.ImmutableList<RexLiteral> constants;
+
+    /**
+     * Creates a window relational expression.
+     *
+     * @param cluster Cluster
+     * @param traitSet Trait set
+     * @param input Input relational expression
+     * @param constants List of constants that are additional inputs
+     * @param rowType Output row type
+     * @param groups Windows
+     */
+    public Window(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode input,
+            List<RexLiteral> constants,
+            RelDataType rowType,
+            List<Group> groups) {
+        super(cluster, traitSet, input);
+        this.constants = com.google.common.collect.ImmutableList.copyOf(constants);
+        assert rowType != null;
+        this.rowType = rowType;
+        this.groups = com.google.common.collect.ImmutableList.copyOf(groups);
+    }
+
+    @Override
+    public boolean isValid(Litmus litmus, RelNode.Context context) {
+        // In the window specifications, an aggregate call such as
+        // 'SUM(RexInputRef #10)' refers to expression #10 of inputProgram.
+        // (Not its projections.)
+        final RelDataType childRowType = getInput().getRowType();
+
+        final int childFieldCount = childRowType.getFieldCount();
+        final int inputSize = childFieldCount + constants.size();
+        final List<RelDataType> inputTypes =
+                new AbstractList<RelDataType>() {
+                    @Override
+                    public RelDataType get(int index) {
+                        return index < childFieldCount
+                                ? childRowType.getFieldList().get(index).getType()
+                                : constants.get(index - childFieldCount).getType();
+                    }
+
+                    @Override
+                    public int size() {
+                        return inputSize;
+                    }
+                };
+
+        final RexChecker checker = new RexChecker(inputTypes, context, litmus);
+        int count = 0;
+        for (Group group : groups) {
+            for (RexWinAggCall over : group.aggCalls) {
+                ++count;
+                if (!checker.isValid(over)) {
+                    return litmus.fail(null);
+                }
+            }
+        }
+        if (count == 0) {
+            return litmus.fail("empty");
+        }
+        return litmus.succeed();
+    }
+
+    public RelWriter explainTerms(RelWriter pw) {
+        super.explainTerms(pw);
+        for (Ord<Group> window : Ord.zip(groups)) {
+            pw.item("window#" + window.i, window.e.toString());
+        }
+        return pw;
+    }
+
+    public static ImmutableIntList getProjectOrdinals(final List<RexNode> exprs) {
+        return ImmutableIntList.copyOf(
+                new AbstractList<Integer>() {
+                    public Integer get(int index) {
+                        return ((RexSlot) exprs.get(index)).getIndex();
+                    }
+
+                    public int size() {
+                        return exprs.size();
+                    }
+                });
+    }
+
+    public static RelCollation getCollation(final List<RexFieldCollation> collations) {
+        return RelCollations.of(
+                new AbstractList<RelFieldCollation>() {
+                    public RelFieldCollation get(int index) {
+                        final RexFieldCollation collation = collations.get(index);
+                        return new RelFieldCollation(
+                                ((RexLocalRef) collation.left).getIndex(),
+                                collation.getDirection(),
+                                collation.getNullDirection());
+                    }
+
+                    public int size() {
+                        return collations.size();
+                    }
+                });
+    }
+
+    /**
+     * Returns constants that are additional inputs of current relation.
+     *
+     * @return constants that are additional inputs of current relation
+     */
+    public List<RexLiteral> getConstants() {
+        return constants;
+    }
+
+    @Override
+    public RelOptCost computeSelfCost(RelOptPlanner planner, RelMetadataQuery mq) {
+        // Cost is proportional to the number of rows and the number of
+        // components (groups and aggregate functions). There is
+        // no I/O cost.
+        //
+        // TODO #1. Add memory cost.
+        // TODO #2. MIN and MAX have higher CPU cost than SUM and COUNT.
+        final double rowsIn = mq.getRowCount(getInput());
+        int count = groups.size();
+        for (Group group : groups) {
+            count += group.aggCalls.size();
+        }
+        return planner.getCostFactory().makeCost(rowsIn, rowsIn * count, 0);
+    }
+
+    /**
+     * Group of windowed aggregate calls that have the same window specification.
+     *
+     * <p>The specification is defined by an upper and lower bound, and also has zero or more
+     * partitioning columns.
+     *
+     * <p>A window is either logical or physical. A physical window is measured in terms of row
+     * count. A logical window is measured in terms of rows within a certain distance from the
+     * current sort key.
+     *
+     * <p>For example:
+     *
+     * <ul>
+     *   <li><code>ROWS BETWEEN 10 PRECEDING and 5 FOLLOWING</code> is a physical window with an
+     *       upper and lower bound;
+     *   <li><code>RANGE BETWEEN INTERVAL '1' HOUR PRECEDING AND UNBOUNDED
+     * FOLLOWING</code> is a logical window with only a lower bound;
+     *   <li><code>RANGE INTERVAL '10' MINUTES PRECEDING</code> (which is equivalent to <code>
+     *       RANGE BETWEEN INTERVAL '10' MINUTES PRECEDING AND
+     * CURRENT ROW</code>) is a logical window with an upper and lower bound.
+     * </ul>
+     */
+    public static class Group {
+        public final ImmutableBitSet keys;
+        public final boolean isRows;
+        public final RexWindowBound lowerBound;
+        public final RexWindowBound upperBound;
+        public final RelCollation orderKeys;
+        private final String digest;
+
+        /**
+         * List of {@link RexWinAggCall} objects, each of which is a call to a {@link
+         * SqlAggFunction}.
+         */
+        public final com.google.common.collect.ImmutableList<RexWinAggCall> aggCalls;
+
+        public Group(
+                ImmutableBitSet keys,
+                boolean isRows,
+                RexWindowBound lowerBound,
+                RexWindowBound upperBound,
+                RelCollation orderKeys,
+                List<RexWinAggCall> aggCalls) {
+            this.keys = Objects.requireNonNull(keys);
+            this.isRows = isRows;
+            this.lowerBound = Objects.requireNonNull(lowerBound);
+            this.upperBound = Objects.requireNonNull(upperBound);
+            this.orderKeys = Objects.requireNonNull(orderKeys);
+            this.aggCalls = com.google.common.collect.ImmutableList.copyOf(aggCalls);
+            this.digest = computeString();
+        }
+
+        public String toString() {
+            return digest;
+        }
+
+        private String computeString() {
+            final StringBuilder buf = new StringBuilder("window(");
+            final int i = buf.length();
+            if (!keys.isEmpty()) {
+                buf.append("partition ");
+                buf.append(keys);
+            }
+            if (!orderKeys.getFieldCollations().isEmpty()) {
+                buf.append(buf.length() == i ? "order by " : " order by ");
+                buf.append(orderKeys);
+            }
+            if (orderKeys.getFieldCollations().isEmpty()
+                    && lowerBound.isUnbounded()
+                    && lowerBound.isPreceding()
+                    && upperBound.isUnbounded()
+                    && upperBound.isFollowing()) {
+                // skip bracket if no ORDER BY, and if bracket is the default,
+                // "RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING",
+                // which is equivalent to
+                // "ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING"
+            } else if (!orderKeys.getFieldCollations().isEmpty()
+                    && lowerBound.isUnbounded()
+                    && lowerBound.isPreceding()
+                    && upperBound.isCurrentRow()
+                    && !isRows) {
+                // skip bracket if there is ORDER BY, and if bracket is the default,
+                // "RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW",
+                // which is NOT equivalent to
+                // "ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW"
+            } else {
+                buf.append(isRows ? " rows " : " range ");
+                buf.append("between ");
+                buf.append(lowerBound);
+                buf.append(" and ");
+                buf.append(upperBound);
+            }
+            if (!aggCalls.isEmpty()) {
+                buf.append(buf.length() == i ? "aggs " : " aggs ");
+                buf.append(aggCalls);
+            }
+            buf.append(")");
+            return buf.toString();
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            return this == obj || obj instanceof Group && this.digest.equals(((Group) obj).digest);
+        }
+
+        @Override
+        public int hashCode() {
+            return digest.hashCode();
+        }
+
+        public RelCollation collation() {
+            return orderKeys;
+        }
+
+        /**
+         * Returns if the window is guaranteed to have rows. This is useful to refine data type of
+         * window aggregates. For instance sum(non-nullable) over (empty window) is NULL.
+         *
+         * @return true when the window is non-empty
+         * @see org.apache.calcite.sql.SqlWindow#isAlwaysNonEmpty()
+         * @see org.apache.calcite.sql.SqlOperatorBinding#getGroupCount()
+         * @see
+         *     org.apache.calcite.sql.validate.SqlValidatorImpl#resolveWindow(org.apache.calcite.sql.SqlNode,
+         *     org.apache.calcite.sql.validate.SqlValidatorScope)
+         */
+        public boolean isAlwaysNonEmpty() {
+            int lowerKey = lowerBound.getOrderKey();
+            int upperKey = upperBound.getOrderKey();
+            return lowerKey > -1 && lowerKey <= upperKey;
+        }
+
+        /** Presents a view of the {@link RexWinAggCall} list as a list of {@link AggregateCall}. */
+        public List<AggregateCall> getAggregateCalls(Window windowRel) {
+            final List<String> fieldNames =
+                    Util.skip(
+                            windowRel.getRowType().getFieldNames(),
+                            windowRel.getInput().getRowType().getFieldCount());
+            return new AbstractList<AggregateCall>() {
+                public int size() {
+                    return aggCalls.size();
+                }
+
+                public AggregateCall get(int index) {
+                    final RexWinAggCall aggCall = aggCalls.get(index);
+                    final SqlAggFunction op = (SqlAggFunction) aggCall.getOperator();
+                    return AggregateCall.create(
+                            op,
+                            aggCall.distinct,
+                            false,
+                            aggCall.ignoreNulls,
+                            getProjectOrdinals(aggCall.getOperands()),
+                            -1,
+                            RelCollations.EMPTY,
+                            aggCall.getType(),
+                            fieldNames.get(aggCall.ordinal));
+                }
+            };
+        }
+    }
+
+    /**
+     * A call to a windowed aggregate function.
+     *
+     * <p>Belongs to a {@link Group}.
+     *
+     * <p>It's a bastard son of a {@link RexCall}; similar enough that it gets visited by a {@link
+     * org.apache.calcite.rex.RexVisitor}, but it also has some extra data members.
+     */
+    public static class RexWinAggCall extends RexCall {
+        /** Ordinal of this aggregate within its partition. */
+        public final int ordinal;
+
+        /** Whether to eliminate duplicates before applying aggregate function. */
+        public final boolean distinct;
+
+        /** Whether to ignore nulls. */
+        public final boolean ignoreNulls;
+
+        @Deprecated // to be removed before 2.0
+        public RexWinAggCall(
+                SqlAggFunction aggFun,
+                RelDataType type,
+                List<RexNode> operands,
+                int ordinal,
+                boolean distinct) {
+            this(aggFun, type, operands, ordinal, distinct, false);
+        }
+
+        /**
+         * Creates a RexWinAggCall.
+         *
+         * @param aggFun Aggregate function
+         * @param type Result type
+         * @param operands Operands to call
+         * @param ordinal Ordinal within its partition
+         * @param distinct Eliminate duplicates before applying aggregate function
+         */
+        public RexWinAggCall(
+                SqlAggFunction aggFun,
+                RelDataType type,
+                List<RexNode> operands,
+                int ordinal,
+                boolean distinct,
+                boolean ignoreNulls) {
+            super(type, aggFun, operands);
+            this.ordinal = ordinal;
+            this.distinct = distinct;
+            this.ignoreNulls = ignoreNulls;
+        }
+
+        @Override
+        public boolean equals(Object o) {
+            if (this == o) {
+                return true;
+            }
+            if (o == null || getClass() != o.getClass()) {
+                return false;
+            }
+            if (!super.equals(o)) {
+                return false;
+            }
+            RexWinAggCall that = (RexWinAggCall) o;
+            return ordinal == that.ordinal
+                    && distinct == that.distinct
+                    && ignoreNulls == that.ignoreNulls;
+        }
+
+        @Override
+        public int hashCode() {
+            if (hash == 0) {
+                hash = Objects.hash(super.hashCode(), ordinal, distinct, ignoreNulls);
+            }
+            return hash;
+        }
+
+        @Override
+        public RexCall clone(RelDataType type, List<RexNode> operands) {
+            return super.clone(type, operands);
+        }
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/HintPredicates.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/HintPredicates.java
new file mode 100644
index 00000000000..59b53e3e089
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/HintPredicates.java
@@ -0,0 +1,86 @@
+/*
+ * 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.calcite.rel.hint;
+
+/** A collection of hint predicates. */
+public abstract class HintPredicates {
+    /**
+     * A hint predicate that indicates a hint can only be used to the whole query(no specific
+     * nodes).
+     */
+    public static final HintPredicate SET_VAR =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.SET_VAR);
+
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.Join} nodes.
+     */
+    public static final HintPredicate JOIN =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.JOIN);
+
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.TableScan} nodes.
+     */
+    public static final HintPredicate TABLE_SCAN =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.TABLE_SCAN);
+
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.Project} nodes.
+     */
+    public static final HintPredicate PROJECT =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.PROJECT);
+
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.Aggregate} nodes.
+     */
+    public static final HintPredicate AGGREGATE =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.AGGREGATE);
+
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.Calc} nodes.
+     */
+    public static final HintPredicate CALC =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.CALC);
+
+    /**
+     * Returns a composed hint predicate that represents a short-circuiting logical AND of an array
+     * of hint predicates {@code hintPredicates}. When evaluating the composed predicate, if a
+     * predicate is {@code false}, then all the left predicates are not evaluated.
+     *
+     * <p>The predicates are evaluated in sequence.
+     */
+    public static HintPredicate and(HintPredicate... hintPredicates) {
+        return new CompositeHintPredicate(CompositeHintPredicate.Composition.AND, hintPredicates);
+    }
+
+    /**
+     * Returns a composed hint predicate that represents a short-circuiting logical OR of an array
+     * of hint predicates {@code hintPredicates}. When evaluating the composed predicate, if a
+     * predicate is {@code true}, then all the left predicates are not evaluated.
+     *
+     * <p>The predicates are evaluated in sequence.
+     */
+    public static HintPredicate or(HintPredicate... hintPredicates) {
+        return new CompositeHintPredicate(CompositeHintPredicate.Composition.OR, hintPredicates);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/NodeTypeHintPredicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/NodeTypeHintPredicate.java
new file mode 100644
index 00000000000..9a5bbff528b
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/NodeTypeHintPredicate.java
@@ -0,0 +1,80 @@
+/*
+ * 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.calcite.rel.hint;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.TableScan;
+
+/**
+ * A hint predicate that specifies which kind of relational expression the hint can be applied to.
+ */
+public class NodeTypeHintPredicate implements HintPredicate {
+
+    /** Enumeration of the relational expression types that the hints may be propagated to. */
+    enum NodeType {
+        /**
+         * The hint is used for the whole query, kind of like a query config. This kind of hints
+         * would never be propagated.
+         */
+        SET_VAR(RelNode.class),
+
+        /** The hint would be propagated to the Join nodes. */
+        JOIN(Join.class),
+
+        /** The hint would be propagated to the TableScan nodes. */
+        TABLE_SCAN(TableScan.class),
+
+        /** The hint would be propagated to the Project nodes. */
+        PROJECT(Project.class),
+
+        /** The hint would be propagated to the Aggregate nodes. */
+        AGGREGATE(Aggregate.class),
+
+        /** The hint would be propagated to the Calc nodes. */
+        CALC(Calc.class);
+
+        /** Relational expression clazz that the hint can apply to. */
+        private Class<?> relClazz;
+
+        NodeType(Class<?> relClazz) {
+            this.relClazz = relClazz;
+        }
+    }
+
+    private NodeType nodeType;
+
+    public NodeTypeHintPredicate(NodeType nodeType) {
+        this.nodeType = nodeType;
+    }
+
+    @Override
+    public boolean apply(RelHint hint, RelNode rel) {
+        switch (this.nodeType) {
+                // Hints of SET_VAR type never propagate.
+            case SET_VAR:
+                return false;
+            default:
+                return this.nodeType.relClazz.isInstance(rel);
+        }
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
new file mode 100644
index 00000000000..01f00b945f9
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
@@ -0,0 +1,116 @@
+/*
+ * 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.calcite.rel.logical;
+
+import org.apache.calcite.config.CalciteSystemProperty;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
+
+/**
+ * A relational operator that performs nested-loop joins.
+ *
+ * <p>It behaves like a kind of {@link org.apache.calcite.rel.core.Join}, but works by setting
+ * variables in its environment and restarting its right-hand input.
+ *
+ * <p>A LogicalCorrelate is used to represent a correlated query. One implementation strategy is to
+ * de-correlate the expression.
+ *
+ * @see CorrelationId
+ */
+public final class LogicalCorrelate extends Correlate {
+    // ~ Instance fields --------------------------------------------------------
+
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a LogicalCorrelate.
+     *
+     * @param cluster cluster this relational expression belongs to
+     * @param left left input relational expression
+     * @param right right input relational expression
+     * @param correlationId variable name for the row of left input
+     * @param requiredColumns Required columns
+     * @param joinType join type
+     */
+    public LogicalCorrelate(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode left,
+            RelNode right,
+            CorrelationId correlationId,
+            ImmutableBitSet requiredColumns,
+            JoinRelType joinType) {
+        super(cluster, traitSet, left, right, correlationId, requiredColumns, joinType);
+        assert !CalciteSystemProperty.DEBUG.value() || isValid(Litmus.THROW, null);
+    }
+
+    /** Creates a LogicalCorrelate by parsing serialized output. */
+    public LogicalCorrelate(RelInput input) {
+        this(
+                input.getCluster(),
+                input.getTraitSet(),
+                input.getInputs().get(0),
+                input.getInputs().get(1),
+                new CorrelationId((Integer) input.get("correlation")),
+                input.getBitSet("requiredColumns"),
+                input.getEnum("joinType", JoinRelType.class));
+    }
+
+    /** Creates a LogicalCorrelate. */
+    public static LogicalCorrelate create(
+            RelNode left,
+            RelNode right,
+            CorrelationId correlationId,
+            ImmutableBitSet requiredColumns,
+            JoinRelType joinType) {
+        final RelOptCluster cluster = left.getCluster();
+        final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE);
+        return new LogicalCorrelate(
+                cluster, traitSet, left, right, correlationId, requiredColumns, joinType);
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    @Override
+    public LogicalCorrelate copy(
+            RelTraitSet traitSet,
+            RelNode left,
+            RelNode right,
+            CorrelationId correlationId,
+            ImmutableBitSet requiredColumns,
+            JoinRelType joinType) {
+        assert traitSet.containsIfApplicable(Convention.NONE);
+        return new LogicalCorrelate(
+                getCluster(), traitSet, left, right, correlationId, requiredColumns, joinType);
+    }
+
+    @Override
+    public RelNode accept(RelShuttle shuttle) {
+        return shuttle.visit(this);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
new file mode 100644
index 00000000000..93f3e61bcbd
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
@@ -0,0 +1,145 @@
+/*
+ * 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.calcite.rel.logical;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelDistributionTraitDef;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.metadata.RelMdCollation;
+import org.apache.calcite.rel.metadata.RelMdDistribution;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.util.Litmus;
+
+import java.util.Objects;
+import java.util.Set;
+
+/** Sub-class of {@link Filter} not targeted at any particular engine or calling convention. */
+public final class LogicalFilter extends Filter {
+    private final com.google.common.collect.ImmutableSet<CorrelationId> variablesSet;
+
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a LogicalFilter.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     *
+     * @param cluster Cluster that this relational expression belongs to
+     * @param child Input relational expression
+     * @param condition Boolean expression which determines whether a row is allowed to pass
+     * @param variablesSet Correlation variables set by this relational expression to be used by
+     *     nested expressions
+     */
+    public LogicalFilter(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode child,
+            RexNode condition,
+            com.google.common.collect.ImmutableSet<CorrelationId> variablesSet) {
+        super(cluster, traitSet, child, condition);
+        this.variablesSet = Objects.requireNonNull(variablesSet);
+        assert isValid(Litmus.THROW, null);
+    }
+
+    @Deprecated // to be removed before 2.0
+    public LogicalFilter(
+            RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode condition) {
+        this(cluster, traitSet, child, condition, com.google.common.collect.ImmutableSet.of());
+    }
+
+    @Deprecated // to be removed before 2.0
+    public LogicalFilter(RelOptCluster cluster, RelNode child, RexNode condition) {
+        this(
+                cluster,
+                cluster.traitSetOf(Convention.NONE),
+                child,
+                condition,
+                com.google.common.collect.ImmutableSet.of());
+    }
+
+    /** Creates a LogicalFilter by parsing serialized output. */
+    public LogicalFilter(RelInput input) {
+        super(input);
+        this.variablesSet = com.google.common.collect.ImmutableSet.of();
+    }
+
+    /** Creates a LogicalFilter. */
+    public static LogicalFilter create(final RelNode input, RexNode condition) {
+        return create(input, condition, com.google.common.collect.ImmutableSet.of());
+    }
+
+    /** Creates a LogicalFilter. */
+    public static LogicalFilter create(
+            final RelNode input,
+            RexNode condition,
+            com.google.common.collect.ImmutableSet<CorrelationId> variablesSet) {
+        final RelOptCluster cluster = input.getCluster();
+        final RelMetadataQuery mq = cluster.getMetadataQuery();
+        final RelTraitSet traitSet =
+                cluster.traitSetOf(Convention.NONE)
+                        .replaceIfs(
+                                RelCollationTraitDef.INSTANCE,
+                                () -> RelMdCollation.filter(mq, input))
+                        .replaceIf(
+                                RelDistributionTraitDef.INSTANCE,
+                                () -> RelMdDistribution.filter(mq, input));
+        return new LogicalFilter(cluster, traitSet, input, condition, variablesSet);
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    @Override
+    public Set<CorrelationId> getVariablesSet() {
+        return variablesSet;
+    }
+
+    public LogicalFilter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
+        assert traitSet.containsIfApplicable(Convention.NONE);
+        return new LogicalFilter(getCluster(), traitSet, input, condition, variablesSet);
+    }
+
+    @Override
+    public RelNode accept(RelShuttle shuttle) {
+        return shuttle.visit(this);
+    }
+
+    @Override
+    public RelWriter explainTerms(RelWriter pw) {
+        return super.explainTerms(pw).itemIf("variablesSet", variablesSet, !variablesSet.isEmpty());
+    }
+
+    @Override
+    public boolean deepEquals(Object obj) {
+        return deepEquals0(obj) && variablesSet.equals(((LogicalFilter) obj).variablesSet);
+    }
+
+    @Override
+    public int deepHashCode() {
+        return Objects.hash(deepHashCode0(), variablesSet);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
new file mode 100644
index 00000000000..ff5ef05c3e3
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
@@ -0,0 +1,76 @@
+/*
+ * 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.calcite.rel.logical;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Intersect;
+
+import java.util.List;
+
+/**
+ * Sub-class of {@link org.apache.calcite.rel.core.Intersect} not targeted at any particular engine
+ * or calling convention.
+ */
+public final class LogicalIntersect extends Intersect {
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a LogicalIntersect.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     */
+    public LogicalIntersect(
+            RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+        super(cluster, traitSet, inputs, all);
+    }
+
+    @Deprecated // to be removed before 2.0
+    public LogicalIntersect(RelOptCluster cluster, List<RelNode> inputs, boolean all) {
+        this(cluster, cluster.traitSetOf(Convention.NONE), inputs, all);
+    }
+
+    /** Creates a LogicalIntersect by parsing serialized output. */
+    public LogicalIntersect(RelInput input) {
+        super(input);
+    }
+
+    /** Creates a LogicalIntersect. */
+    public static LogicalIntersect create(List<RelNode> inputs, boolean all) {
+        final RelOptCluster cluster = inputs.get(0).getCluster();
+        final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE);
+        return new LogicalIntersect(cluster, traitSet, inputs, all);
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    @Override
+    public LogicalIntersect copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+        return new LogicalIntersect(getCluster(), traitSet, inputs, all);
+    }
+
+    @Override
+    public RelNode accept(RelShuttle shuttle) {
+        return shuttle.visit(this);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
new file mode 100644
index 00000000000..ea2eb044156
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
@@ -0,0 +1,74 @@
+/*
+ * 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.calcite.rel.logical;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Minus;
+
+import java.util.List;
+
+/** Sub-class of {@link Minus} not targeted at any particular engine or calling convention. */
+public final class LogicalMinus extends Minus {
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a LogicalMinus.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     */
+    public LogicalMinus(
+            RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+        super(cluster, traitSet, inputs, all);
+    }
+
+    @Deprecated // to be removed before 2.0
+    public LogicalMinus(RelOptCluster cluster, List<RelNode> inputs, boolean all) {
+        this(cluster, cluster.traitSetOf(Convention.NONE), inputs, all);
+    }
+
+    /** Creates a LogicalMinus by parsing serialized output. */
+    public LogicalMinus(RelInput input) {
+        super(input);
+    }
+
+    /** Creates a LogicalMinus. */
+    public static LogicalMinus create(List<RelNode> inputs, boolean all) {
+        final RelOptCluster cluster = inputs.get(0).getCluster();
+        final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE);
+        return new LogicalMinus(cluster, traitSet, inputs, all);
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    @Override
+    public LogicalMinus copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+        assert traitSet.containsIfApplicable(Convention.NONE);
+        return new LogicalMinus(getCluster(), traitSet, inputs, all);
+    }
+
+    @Override
+    public RelNode accept(RelShuttle shuttle) {
+        return shuttle.visit(this);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalSort.java
new file mode 100644
index 00000000000..437744ee039
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalSort.java
@@ -0,0 +1,82 @@
+/*
+ * 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.calcite.rel.logical;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rex.RexNode;
+
+/** Sub-class of {@link Sort} not targeted at any particular engine or calling convention. */
+public final class LogicalSort extends Sort {
+    private LogicalSort(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode input,
+            RelCollation collation,
+            RexNode offset,
+            RexNode fetch) {
+        super(cluster, traitSet, input, collation, offset, fetch);
+        assert traitSet.containsIfApplicable(Convention.NONE);
+    }
+
+    /** Creates a LogicalSort by parsing serialized output. */
+    public LogicalSort(RelInput input) {
+        super(input);
+    }
+
+    /**
+     * Creates a LogicalSort.
+     *
+     * @param input Input relational expression
+     * @param collation array of sort specifications
+     * @param offset Expression for number of rows to discard before returning first row
+     * @param fetch Expression for number of rows to fetch
+     */
+    public static LogicalSort create(
+            RelNode input, RelCollation collation, RexNode offset, RexNode fetch) {
+        RelOptCluster cluster = input.getCluster();
+        collation = RelCollationTraitDef.INSTANCE.canonize(collation);
+        RelTraitSet traitSet = input.getTraitSet().replace(Convention.NONE).replace(collation);
+        return new LogicalSort(cluster, traitSet, input, collation, offset, fetch);
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    @Override
+    public Sort copy(
+            RelTraitSet traitSet,
+            RelNode newInput,
+            RelCollation newCollation,
+            RexNode offset,
+            RexNode fetch) {
+        return new LogicalSort(getCluster(), traitSet, newInput, newCollation, offset, fetch);
+    }
+
+    @Override
+    public RelNode accept(RelShuttle shuttle) {
+        return shuttle.visit(this);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
new file mode 100644
index 00000000000..25b4420c188
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
@@ -0,0 +1,73 @@
+/*
+ * 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.calcite.rel.logical;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Union;
+
+import java.util.List;
+
+/** Sub-class of {@link Union} not targeted at any particular engine or calling convention. */
+public final class LogicalUnion extends Union {
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a LogicalUnion.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     */
+    public LogicalUnion(
+            RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+        super(cluster, traitSet, inputs, all);
+    }
+
+    @Deprecated // to be removed before 2.0
+    public LogicalUnion(RelOptCluster cluster, List<RelNode> inputs, boolean all) {
+        this(cluster, cluster.traitSetOf(Convention.NONE), inputs, all);
+    }
+
+    /** Creates a LogicalUnion by parsing serialized output. */
+    public LogicalUnion(RelInput input) {
+        super(input);
+    }
+
+    /** Creates a LogicalUnion. */
+    public static LogicalUnion create(List<RelNode> inputs, boolean all) {
+        final RelOptCluster cluster = inputs.get(0).getCluster();
+        final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE);
+        return new LogicalUnion(cluster, traitSet, inputs, all);
+    }
+
+    // ~ Methods ----------------------------------------------------------------
+
+    public LogicalUnion copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
+        assert traitSet.containsIfApplicable(Convention.NONE);
+        return new LogicalUnion(getCluster(), traitSet, inputs, all);
+    }
+
+    @Override
+    public RelNode accept(RelShuttle shuttle) {
+        return shuttle.visit(this);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
new file mode 100644
index 00000000000..c8ff3179c7f
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
@@ -0,0 +1,134 @@
+/*
+ * 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.calcite.rel.logical;
+
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.calcite.rel.RelDistributionTraitDef;
+import org.apache.calcite.rel.RelInput;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.metadata.RelMdCollation;
+import org.apache.calcite.rel.metadata.RelMdDistribution;
+import org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+import java.math.BigDecimal;
+import java.util.List;
+
+/** Sub-class of {@link Values} not targeted at any particular engine or calling convention. */
+public class LogicalValues extends Values {
+    // ~ Constructors -----------------------------------------------------------
+
+    /**
+     * Creates a LogicalValues.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     *
+     * @param cluster Cluster that this relational expression belongs to
+     * @param rowType Row type for tuples produced by this rel
+     * @param tuples 2-dimensional array of tuple values to be produced; outer list contains tuples;
+     *     each inner list is one tuple; all tuples must be of same length, conforming to rowType
+     */
+    public LogicalValues(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelDataType rowType,
+            com.google.common.collect.ImmutableList<
+                            com.google.common.collect.ImmutableList<RexLiteral>>
+                    tuples) {
+        super(cluster, rowType, tuples, traitSet);
+    }
+
+    @Deprecated // to be removed before 2.0
+    public LogicalValues(
+            RelOptCluster cluster,
+            RelDataType rowType,
+            com.google.common.collect.ImmutableList<
+                            com.google.common.collect.ImmutableList<RexLiteral>>
+                    tuples) {
+        this(cluster, cluster.traitSetOf(Convention.NONE), rowType, tuples);
+    }
+
+    /** Creates a LogicalValues by parsing serialized output. */
+    public LogicalValues(RelInput input) {
+        super(input);
+    }
+
+    /** Creates a LogicalValues. */
+    public static LogicalValues create(
+            RelOptCluster cluster,
+            final RelDataType rowType,
+            final com.google.common.collect.ImmutableList<
+                            com.google.common.collect.ImmutableList<RexLiteral>>
+                    tuples) {
+        final RelMetadataQuery mq = cluster.getMetadataQuery();
+        final RelTraitSet traitSet =
+                cluster.traitSetOf(Convention.NONE)
+                        .replaceIfs(
+                                RelCollationTraitDef.INSTANCE,
+                                () -> RelMdCollation.values(mq, rowType, tuples))
+                        .replaceIf(
+                                RelDistributionTraitDef.INSTANCE,
+                                () -> RelMdDistribution.values(rowType, tuples));
+        return new LogicalValues(cluster, traitSet, rowType, tuples);
+    }
+
+    @Override
+    public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        assert traitSet.containsIfApplicable(Convention.NONE);
+        assert inputs.isEmpty();
+        return new LogicalValues(getCluster(), traitSet, rowType, tuples);
+    }
+
+    /** Creates a LogicalValues that outputs no rows of a given row type. */
+    public static LogicalValues createEmpty(RelOptCluster cluster, RelDataType rowType) {
+        return create(cluster, rowType, com.google.common.collect.ImmutableList.of());
+    }
+
+    /** Creates a LogicalValues that outputs one row and one column. */
+    public static LogicalValues createOneRow(RelOptCluster cluster) {
+        final RelDataType rowType =
+                cluster.getTypeFactory()
+                        .builder()
+                        .add("ZERO", SqlTypeName.INTEGER)
+                        .nullable(false)
+                        .build();
+        final com.google.common.collect.ImmutableList<
+                        com.google.common.collect.ImmutableList<RexLiteral>>
+                tuples =
+                        com.google.common.collect.ImmutableList.of(
+                                com.google.common.collect.ImmutableList.of(
+                                        cluster.getRexBuilder()
+                                                .makeExactLiteral(
+                                                        BigDecimal.ZERO,
+                                                        rowType.getFieldList().get(0).getType())));
+        return create(cluster, rowType, tuples);
+    }
+
+    @Override
+    public RelNode accept(RelShuttle shuttle) {
+        return shuttle.visit(this);
+    }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
new file mode 100644
index 00000000000..ef5f87092c0
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
@@ -0,0 +1,367 @@
+/*
+ * 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.calcite.rel.logical;
+
+import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelCollation;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Window;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexProgram;
+import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.rex.RexWindow;
+import org.apache.calcite.rex.RexWindowBound;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
+
+import java.util.AbstractList;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.IdentityHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/** Sub-class of {@link Window} not targeted at any particular engine or calling convention. */
+public final class LogicalWindow extends Window {
+    /**
+     * Creates a LogicalWindow.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     *
+     * @param cluster Cluster
+     * @param traitSet Trait set
+     * @param input Input relational expression
+     * @param constants List of constants that are additional inputs
+     * @param rowType Output row type
+     * @param groups Window groups
+     */
+    public LogicalWindow(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode input,
+            List<RexLiteral> constants,
+            RelDataType rowType,
+            List<Group> groups) {
+        super(cluster, traitSet, input, constants, rowType, groups);
+    }
+
+    @Override
+    public LogicalWindow copy(RelTraitSet traitSet, List<RelNode> inputs) {
+        return new LogicalWindow(getCluster(), traitSet, sole(inputs), constants, rowType, groups);
+    }
+
+    /**
+     * Creates a LogicalWindow.
+     *
+     * @param input Input relational expression
+     * @param traitSet Trait set
+     * @param constants List of constants that are additional inputs
+     * @param rowType Output row type
+     * @param groups Window groups
+     */
+    public static LogicalWindow create(
+            RelTraitSet traitSet,
+            RelNode input,
+            List<RexLiteral> constants,
+            RelDataType rowType,
+            List<Window.Group> groups) {
+        return new LogicalWindow(input.getCluster(), traitSet, input, constants, rowType, groups);
+    }
+
+    /** Creates a LogicalWindow by parsing a {@link RexProgram}. */
+    public static RelNode create(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelBuilder relBuilder,
+            RelNode child,
+            final RexProgram program) {
+        final RelDataType outRowType = program.getOutputRowType();
+        // Build a list of distinct groups, partitions and aggregate
+        // functions.
+        final com.google.common.collect.Multimap<WindowKey, RexOver> windowMap =
+                com.google.common.collect.LinkedListMultimap.create();
+
+        final int inputFieldCount = child.getRowType().getFieldCount();
+
+        final Map<RexLiteral, RexInputRef> constantPool = new HashMap<>();
+        final List<RexLiteral> constants = new ArrayList<>();
+
+        // Identify constants in the expression tree and replace them with
+        // references to newly generated constant pool.
+        RexShuttle replaceConstants =
+                new RexShuttle() {
+                    @Override
+                    public RexNode visitLiteral(RexLiteral literal) {
+                        RexInputRef ref = constantPool.get(literal);
+                        if (ref != null) {
+                            return ref;
+                        }
+                        constants.add(literal);
+                        ref =
+                                new RexInputRef(
+                                        constantPool.size() + inputFieldCount, literal.getType());
+                        constantPool.put(literal, ref);
+                        return ref;
+                    }
+                };
+
+        // Build a list of groups, partitions, and aggregate functions. Each
+        // aggregate function will add its arguments as outputs of the input
+        // program.
+        final Map<RexOver, RexOver> origToNewOver = new IdentityHashMap<>();
+        for (RexNode agg : program.getExprList()) {
+            if (agg instanceof RexOver) {
+                final RexOver origOver = (RexOver) agg;
+                final RexOver newOver = (RexOver) origOver.accept(replaceConstants);
+                origToNewOver.put(origOver, newOver);
+                addWindows(windowMap, newOver, inputFieldCount);
+            }
+        }
+
+        final Map<RexOver, RexWinAggCall> aggMap = new HashMap<>();
+        List<Group> groups = new ArrayList<>();
+        for (Map.Entry<WindowKey, Collection<RexOver>> entry : windowMap.asMap().entrySet()) {
+            final WindowKey windowKey = entry.getKey();
+            final List<RexWinAggCall> aggCalls = new ArrayList<>();
+            for (RexOver over : entry.getValue()) {
+                final RexWinAggCall aggCall =
+                        new RexWinAggCall(
+                                over.getAggOperator(),
+                                over.getType(),
+                                toInputRefs(over.operands),
+                                aggMap.size(),
+                                over.isDistinct(),
+                                over.ignoreNulls());
+                aggCalls.add(aggCall);
+                aggMap.put(over, aggCall);
+            }
+            RexShuttle toInputRefs =
+                    new RexShuttle() {
+                        @Override
+                        public RexNode visitLocalRef(RexLocalRef localRef) {
+                            return new RexInputRef(localRef.getIndex(), localRef.getType());
+                        }
+                    };
+            groups.add(
+                    new Group(
+                            windowKey.groupSet,
+                            windowKey.isRows,
+                            windowKey.lowerBound.accept(toInputRefs),
+                            windowKey.upperBound.accept(toInputRefs),
+                            windowKey.orderKeys,
+                            aggCalls));
+        }
+
+        // Figure out the type of the inputs to the output program.
+        // They are: the inputs to this rel, followed by the outputs of
+        // each window.
+        final List<RexWinAggCall> flattenedAggCallList = new ArrayList<>();
+        final List<Map.Entry<String, RelDataType>> fieldList =
+                new ArrayList<>(child.getRowType().getFieldList());
+        final int offset = fieldList.size();
+
+        // Use better field names for agg calls that are projected.
+        final Map<Integer, String> fieldNames = new HashMap<>();
+        for (Ord<RexLocalRef> ref : Ord.zip(program.getProjectList())) {
+            final int index = ref.e.getIndex();
+            if (index >= offset) {
+                fieldNames.put(index - offset, outRowType.getFieldNames().get(ref.i));
+            }
+        }
+
+        for (Ord<Group> window : Ord.zip(groups)) {
+            for (Ord<RexWinAggCall> over : Ord.zip(window.e.aggCalls)) {
+                // Add the k-th over expression of
+                // the i-th window to the output of the program.
+                String name = fieldNames.get(over.i);
+                if (name == null || name.startsWith("$")) {
+                    name = "w" + window.i + "$o" + over.i;
+                }
+                fieldList.add(Pair.of(name, over.e.getType()));
+                flattenedAggCallList.add(over.e);
+            }
+        }
+        final RelDataType intermediateRowType =
+                cluster.getTypeFactory().createStructType(fieldList);
+
+        // The output program is the windowed agg's program, combined with
+        // the output calc (if it exists).
+        RexShuttle shuttle =
+                new RexShuttle() {
+                    public RexNode visitOver(RexOver over) {
+                        // Look up the aggCall which this expr was translated to.
+                        final RexWinAggCall aggCall = aggMap.get(origToNewOver.get(over));
+                        assert aggCall != null;
+                        assert RelOptUtil.eq(
+                                "over", over.getType(), "aggCall", aggCall.getType(), Litmus.THROW);
+
+                        // Find the index of the aggCall among all partitions of all
+                        // groups.
+                        final int aggCallIndex = flattenedAggCallList.indexOf(aggCall);
+                        assert aggCallIndex >= 0;
+
+                        // Replace expression with a reference to the window slot.
+                        final int index = inputFieldCount + aggCallIndex;
+                        assert RelOptUtil.eq(
+                                "over",
+                                over.getType(),
+                                "intermed",
+                                intermediateRowType.getFieldList().get(index).getType(),
+                                Litmus.THROW);
+                        return new RexInputRef(index, over.getType());
+                    }
+
+                    public RexNode visitLocalRef(RexLocalRef localRef) {
+                        final int index = localRef.getIndex();
+                        if (index < inputFieldCount) {
+                            // Reference to input field.
+                            return localRef;
+                        }
+                        return new RexLocalRef(
+                                flattenedAggCallList.size() + index, localRef.getType());
+                    }
+                };
+
+        final LogicalWindow window =
+                LogicalWindow.create(traitSet, child, constants, intermediateRowType, groups);
+
+        // The order that the "over" calls occur in the groups and
+        // partitions may not match the order in which they occurred in the
+        // original expression.
+        // Add a project to permute them.
+        final List<RexNode> refToWindow = toInputRefs(shuttle.visitList(program.getExprList()));
+
+        final List<RexNode> projectList = new ArrayList<>();
+        for (RexLocalRef inputRef : program.getProjectList()) {
+            final int index = inputRef.getIndex();
+            final RexInputRef ref = (RexInputRef) refToWindow.get(index);
+            projectList.add(ref);
+        }
+
+        return relBuilder.push(window).project(projectList, outRowType.getFieldNames()).build();
+    }
+
+    private static List<RexNode> toInputRefs(final List<? extends RexNode> operands) {
+        return new AbstractList<RexNode>() {
+            public int size() {
+                return operands.size();
+            }
+
+            public RexNode get(int index) {
+                final RexNode operand = operands.get(index);
+                if (operand instanceof RexInputRef) {
+                    return operand;
+                }
+                assert operand instanceof RexLocalRef;
+                final RexLocalRef ref = (RexLocalRef) operand;
+                return new RexInputRef(ref.getIndex(), ref.getType());
+            }
+        };
+    }
+
+    /**
+     * Group specification. All windowed aggregates over the same window (regardless of how it is
+     * specified, in terms of a named window or specified attribute by attribute) will end up with
+     * the same window key.
+     */
+    private static class WindowKey {
+        private final ImmutableBitSet groupSet;
+        private final RelCollation orderKeys;
+        private final boolean isRows;
+        private final RexWindowBound lowerBound;
+        private final RexWindowBound upperBound;
+
+        WindowKey(
+                ImmutableBitSet groupSet,
+                RelCollation orderKeys,
+                boolean isRows,
+                RexWindowBound lowerBound,
+                RexWindowBound upperBound) {
+            this.groupSet = groupSet;
+            this.orderKeys = orderKeys;
+            this.isRows = isRows;
+            this.lowerBound = lowerBound;
+            this.upperBound = upperBound;
+        }
+
+        @Override
+        public int hashCode() {
+            return Objects.hash(groupSet, orderKeys, isRows, lowerBound, upperBound);
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            return obj == this
+                    || obj instanceof WindowKey
+                            && groupSet.equals(((WindowKey) obj).groupSet)
+                            && orderKeys.equals(((WindowKey) obj).orderKeys)
+                            && Objects.equals(lowerBound, ((WindowKey) obj).lowerBound)
+                            && Objects.equals(upperBound, ((WindowKey) obj).upperBound)
+                            && isRows == ((WindowKey) obj).isRows;
+        }
+    }
+
+    private static void addWindows(
+            com.google.common.collect.Multimap<WindowKey, RexOver> windowMap,
+            RexOver over,
+            final int inputFieldCount) {
+        final RexWindow aggWindow = over.getWindow();
+
+        // Look up or create a window.
+        RelCollation orderKeys =
+                getCollation(
+                        com.google.common.collect.Lists.newArrayList(
+                                Util.filter(
+                                        aggWindow.orderKeys,
+                                        rexFieldCollation ->
+                                                // If ORDER BY references constant (i.e.
+                                                // RexInputRef),
+                                                // then we can ignore such ORDER BY key.
+                                                rexFieldCollation.left instanceof RexLocalRef)));
+        ImmutableBitSet groupSet = ImmutableBitSet.of(getProjectOrdinals(aggWindow.partitionKeys));
+        final int groupLength = groupSet.length();
+        if (inputFieldCount < groupLength) {
+            // If PARTITION BY references constant, we can ignore such partition key.
+            // All the inputs after inputFieldCount are literals, thus we can clear.
+            groupSet = groupSet.except(ImmutableBitSet.range(inputFieldCount, groupLength));
+        }
+
+        WindowKey windowKey =
+                new WindowKey(
+                        groupSet,
+                        orderKeys,
+                        aggWindow.isRows(),
+                        aggWindow.getLowerBound(),
+                        aggWindow.getUpperBound());
+        windowMap.put(windowKey, over);
+    }
+}


[flink] 02/04: [FLINK-28678][table-planner] Support SQL hint for Filter, SetOp, and etc. Mainly cherry-pick from [calcite-5107]

Posted by go...@apache.org.
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

commit 4e46602342f3e3e45716f3adf8e29f7531cb9ca5
Author: xuyang <xy...@163.com>
AuthorDate: Tue Jul 26 14:25:40 2022 +0800

    [FLINK-28678][table-planner] Support SQL hint for Filter, SetOp, and etc. Mainly cherry-pick from [calcite-5107]
    
    Since currently the feat [CALCITE-5107] is not introduced to the calcite version flink used, so just temporarily cherry-pick this feat to copied files directly. When the calcite upgrades to 1.31, these classes should be deleted.
    
    This closes #20359
---
 .../org/apache/calcite/rel/core/Correlate.java     | 46 ++++++++++++-
 .../java/org/apache/calcite/rel/core/Filter.java   | 38 ++++++++++-
 .../org/apache/calcite/rel/core/Intersect.java     | 17 ++++-
 .../java/org/apache/calcite/rel/core/Minus.java    | 17 ++++-
 .../java/org/apache/calcite/rel/core/SetOp.java    | 27 +++++++-
 .../java/org/apache/calcite/rel/core/Sort.java     | 39 ++++++++++-
 .../java/org/apache/calcite/rel/core/Union.java    | 16 ++++-
 .../java/org/apache/calcite/rel/core/Values.java   | 43 +++++++++++-
 .../java/org/apache/calcite/rel/core/Window.java   | 50 ++++++++++++--
 .../apache/calcite/rel/hint/HintPredicates.java    | 49 +++++++++++++-
 .../calcite/rel/hint/NodeTypeHintPredicate.java    | 29 +++++++-
 .../calcite/rel/logical/LogicalCorrelate.java      | 78 +++++++++++++++++++++-
 .../apache/calcite/rel/logical/LogicalFilter.java  | 36 +++++++++-
 .../calcite/rel/logical/LogicalIntersect.java      | 25 ++++++-
 .../apache/calcite/rel/logical/LogicalMinus.java   | 33 ++++++++-
 .../apache/calcite/rel/logical/LogicalSort.java    | 33 ++++++++-
 .../apache/calcite/rel/logical/LogicalUnion.java   | 33 ++++++++-
 .../apache/calcite/rel/logical/LogicalValues.java  | 39 ++++++++++-
 .../apache/calcite/rel/logical/LogicalWindow.java  | 44 ++++++++++--
 19 files changed, 646 insertions(+), 46 deletions(-)

diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Correlate.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Correlate.java
index 2d5e6fe554f..8df4da8b823 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Correlate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Correlate.java
@@ -27,12 +27,15 @@ import org.apache.calcite.rel.BiRel;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Litmus;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
@@ -40,6 +43,9 @@ import java.util.Set;
 /**
  * A relational operator that performs nested-loop joins.
  *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ *
  * <p>It behaves like a kind of {@link org.apache.calcite.rel.core.Join}, but works by setting
  * variables in its environment and restarting its right-hand input.
  *
@@ -63,12 +69,13 @@ import java.util.Set;
  *
  * @see CorrelationId
  */
-public abstract class Correlate extends BiRel {
+public abstract class Correlate extends BiRel implements Hintable {
     // ~ Instance fields --------------------------------------------------------
 
     protected final CorrelationId correlationId;
     protected final ImmutableBitSet requiredColumns;
     protected final JoinRelType joinType;
+    protected final com.google.common.collect.ImmutableList<RelHint> hints;
 
     // ~ Constructors -----------------------------------------------------------
 
@@ -76,6 +83,7 @@ public abstract class Correlate extends BiRel {
      * Creates a Correlate.
      *
      * @param cluster Cluster this relational expression belongs to
+     * @param hints Hints for this node
      * @param left Left input relational expression
      * @param right Right input relational expression
      * @param correlationId Variable name for the row of left input
@@ -85,6 +93,7 @@ public abstract class Correlate extends BiRel {
     protected Correlate(
             RelOptCluster cluster,
             RelTraitSet traitSet,
+            List<RelHint> hints,
             RelNode left,
             RelNode right,
             CorrelationId correlationId,
@@ -95,6 +104,36 @@ public abstract class Correlate extends BiRel {
         this.joinType = Objects.requireNonNull(joinType);
         this.correlationId = Objects.requireNonNull(correlationId);
         this.requiredColumns = Objects.requireNonNull(requiredColumns);
+        this.hints = com.google.common.collect.ImmutableList.copyOf(hints);
+    }
+
+    /**
+     * Creates a Correlate.
+     *
+     * @param cluster Cluster this relational expression belongs to
+     * @param left Left input relational expression
+     * @param right Right input relational expression
+     * @param correlationId Variable name for the row of left input
+     * @param requiredColumns Set of columns that are used by correlation
+     * @param joinType Join type
+     */
+    protected Correlate(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode left,
+            RelNode right,
+            CorrelationId correlationId,
+            ImmutableBitSet requiredColumns,
+            JoinRelType joinType) {
+        this(
+                cluster,
+                traitSet,
+                Collections.emptyList(),
+                left,
+                right,
+                correlationId,
+                requiredColumns,
+                joinType);
     }
 
     /**
@@ -231,4 +270,9 @@ public abstract class Correlate extends BiRel {
                         0)
                 .plus(rescanCost);
     }
+
+    @Override
+    public com.google.common.collect.ImmutableList<RelHint> getHints() {
+        return hints;
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Filter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Filter.java
index 2fc9f0c1240..34a28def16b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Filter.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Filter.java
@@ -27,6 +27,8 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexChecker;
@@ -44,15 +46,20 @@ import java.util.Objects;
  * Relational expression that iterates over its input and returns elements for which <code>condition
  * </code> evaluates to <code>true</code>.
  *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ *
  * <p>If the condition allows nulls, then a null value is treated the same as false.
  *
  * @see org.apache.calcite.rel.logical.LogicalFilter
  */
-public abstract class Filter extends SingleRel {
+public abstract class Filter extends SingleRel implements Hintable {
     // ~ Instance fields --------------------------------------------------------
 
     protected final RexNode condition;
 
+    protected final com.google.common.collect.ImmutableList<RelHint> hints;
+
     // ~ Constructors -----------------------------------------------------------
 
     /**
@@ -60,16 +67,35 @@ public abstract class Filter extends SingleRel {
      *
      * @param cluster Cluster that this relational expression belongs to
      * @param traits the traits of this rel
+     * @param hints Hints for this node
      * @param child input relational expression
      * @param condition boolean expression which determines whether a row is allowed to pass
      */
-    protected Filter(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+    protected Filter(
+            RelOptCluster cluster,
+            RelTraitSet traits,
+            List<RelHint> hints,
+            RelNode child,
+            RexNode condition) {
         super(cluster, traits, child);
         assert condition != null;
         assert RexUtil.isFlat(condition) : condition;
         this.condition = condition;
         // Too expensive for everyday use:
         assert !CalciteSystemProperty.DEBUG.value() || isValid(Litmus.THROW, null);
+        this.hints = com.google.common.collect.ImmutableList.copyOf(hints);
+    }
+
+    /**
+     * Creates a filter.
+     *
+     * @param cluster Cluster that this relational expression belongs to
+     * @param traits the traits of this rel
+     * @param child input relational expression
+     * @param condition boolean expression which determines whether a row is allowed to pass
+     */
+    protected Filter(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+        this(cluster, traits, com.google.common.collect.ImmutableList.of(), child, condition);
     }
 
     /** Creates a Filter by parsing serialized output. */
@@ -158,12 +184,18 @@ public abstract class Filter extends SingleRel {
         }
         Filter o = (Filter) obj;
         return traitSet.equals(o.traitSet)
+                && hints.equals(o.hints)
                 && input.deepEquals(o.input)
                 && condition.equals(o.condition)
                 && getRowType().equalsSansFieldNames(o.getRowType());
     }
 
     protected int deepHashCode0() {
-        return Objects.hash(traitSet, input.deepHashCode(), condition);
+        return Objects.hash(traitSet, hints, input.deepHashCode(), condition);
+    }
+
+    @Override
+    public com.google.common.collect.ImmutableList<RelHint> getHints() {
+        return hints;
     }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Intersect.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Intersect.java
index c923692275b..ad20e66fe37 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Intersect.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Intersect.java
@@ -22,21 +22,36 @@ import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.sql.SqlKind;
 
+import java.util.Collections;
 import java.util.List;
 
 /**
  * Relational expression that returns the intersection of the rows of its inputs.
  *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ *
  * <p>If "all" is true, performs then multiset intersection; otherwise, performs set set
  * intersection (implying no duplicates in the results).
  */
 public abstract class Intersect extends SetOp {
+    /** Creates an Intersect. */
+    public Intersect(
+            RelOptCluster cluster,
+            RelTraitSet traits,
+            List<RelHint> hints,
+            List<RelNode> inputs,
+            boolean all) {
+        super(cluster, traits, hints, inputs, SqlKind.INTERSECT, all);
+    }
+
     /** Creates an Intersect. */
     public Intersect(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
-        super(cluster, traits, inputs, SqlKind.INTERSECT, all);
+        this(cluster, traits, Collections.emptyList(), inputs, all);
     }
 
     /** Creates an Intersect by parsing serialized output. */
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Minus.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Minus.java
index 035581e4150..ae3cb8c6d23 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Minus.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Minus.java
@@ -22,24 +22,39 @@ import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.sql.SqlKind;
 
+import java.util.Collections;
 import java.util.List;
 
 /**
  * Relational expression that returns the rows of its first input minus any matching rows from its
  * other inputs.
  *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ *
  * <p>Corresponds to the SQL {@code EXCEPT} operator.
  *
  * <p>If "all" is true, then multiset subtraction is performed; otherwise, set subtraction is
  * performed (implying no duplicates in the results).
  */
 public abstract class Minus extends SetOp {
+
+    public Minus(
+            RelOptCluster cluster,
+            RelTraitSet traits,
+            List<RelHint> hints,
+            List<RelNode> inputs,
+            boolean all) {
+        super(cluster, traits, hints, inputs, SqlKind.EXCEPT, all);
+    }
+
     public Minus(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
-        super(cluster, traits, inputs, SqlKind.EXCEPT, all);
+        this(cluster, traits, Collections.emptyList(), inputs, all);
     }
 
     /** Creates a Minus by parsing serialized output. */
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/SetOp.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/SetOp.java
index 2e323031102..710fcaed874 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/SetOp.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/SetOp.java
@@ -26,23 +26,30 @@ import org.apache.calcite.rel.AbstractRelNode;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.util.Util;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 /**
  * <code>SetOp</code> is an abstract base for relational set operators such as UNION, MINUS (aka
  * EXCEPT), and INTERSECT.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
  */
-public abstract class SetOp extends AbstractRelNode {
+public abstract class SetOp extends AbstractRelNode implements Hintable {
     // ~ Instance fields --------------------------------------------------------
 
     protected com.google.common.collect.ImmutableList<RelNode> inputs;
     public final SqlKind kind;
     public final boolean all;
+    protected final com.google.common.collect.ImmutableList<RelHint> hints;
 
     // ~ Constructors -----------------------------------------------------------
 
@@ -50,6 +57,7 @@ public abstract class SetOp extends AbstractRelNode {
     protected SetOp(
             RelOptCluster cluster,
             RelTraitSet traits,
+            List<RelHint> hints,
             List<RelNode> inputs,
             SqlKind kind,
             boolean all) {
@@ -59,6 +67,17 @@ public abstract class SetOp extends AbstractRelNode {
         this.kind = kind;
         this.inputs = com.google.common.collect.ImmutableList.copyOf(inputs);
         this.all = all;
+        this.hints = com.google.common.collect.ImmutableList.copyOf(hints);
+    }
+
+    /** Creates a SetOp. */
+    protected SetOp(
+            RelOptCluster cluster,
+            RelTraitSet traits,
+            List<RelNode> inputs,
+            SqlKind kind,
+            boolean all) {
+        this(cluster, traits, Collections.emptyList(), inputs, kind, all);
     }
 
     /** Creates a SetOp by parsing serialized output. */
@@ -66,6 +85,7 @@ public abstract class SetOp extends AbstractRelNode {
         this(
                 input.getCluster(),
                 input.getTraitSet(),
+                Collections.emptyList(),
                 input.getInputs(),
                 SqlKind.UNION,
                 input.getBoolean("all", false));
@@ -130,4 +150,9 @@ public abstract class SetOp extends AbstractRelNode {
         }
         return true;
     }
+
+    @Override
+    public com.google.common.collect.ImmutableList<RelHint> getHints() {
+        return hints;
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Sort.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Sort.java
index 8ebab6e03be..93c38fbdd44 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Sort.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Sort.java
@@ -30,24 +30,31 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.util.Util;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
 /**
  * Relational expression that imposes a particular sort order on its input without otherwise
  * changing its content.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
  */
-public abstract class Sort extends SingleRel {
+public abstract class Sort extends SingleRel implements Hintable {
     // ~ Instance fields --------------------------------------------------------
 
     public final RelCollation collation;
     public final RexNode offset;
     public final RexNode fetch;
+    protected final com.google.common.collect.ImmutableList<RelHint> hints;
 
     // ~ Constructors -----------------------------------------------------------
 
@@ -60,7 +67,7 @@ public abstract class Sort extends SingleRel {
      * @param collation array of sort specifications
      */
     public Sort(RelOptCluster cluster, RelTraitSet traits, RelNode child, RelCollation collation) {
-        this(cluster, traits, child, collation, null, null);
+        this(cluster, traits, Collections.emptyList(), child, collation, null, null);
     }
 
     /**
@@ -80,10 +87,33 @@ public abstract class Sort extends SingleRel {
             RelCollation collation,
             RexNode offset,
             RexNode fetch) {
+        this(cluster, traits, Collections.emptyList(), child, collation, offset, fetch);
+    }
+
+    /**
+     * Creates a Sort.
+     *
+     * @param cluster Cluster this relational expression belongs to
+     * @param traits Traits
+     * @param hints Hints for this node
+     * @param child input relational expression
+     * @param collation array of sort specifications
+     * @param offset Expression for number of rows to discard before returning first row
+     * @param fetch Expression for number of rows to fetch
+     */
+    public Sort(
+            RelOptCluster cluster,
+            RelTraitSet traits,
+            List<RelHint> hints,
+            RelNode child,
+            RelCollation collation,
+            RexNode offset,
+            RexNode fetch) {
         super(cluster, traits, child);
         this.collation = collation;
         this.offset = offset;
         this.fetch = fetch;
+        this.hints = com.google.common.collect.ImmutableList.copyOf(hints);
 
         assert traits.containsIfApplicable(collation)
                 : "traits=" + traits + ", collation=" + collation;
@@ -194,4 +224,9 @@ public abstract class Sort extends SingleRel {
         pw.itemIf("fetch", fetch, fetch != null);
         return pw;
     }
+
+    @Override
+    public com.google.common.collect.ImmutableList<RelHint> getHints() {
+        return hints;
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Union.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Union.java
index f79538b45e7..f4099ae1f9a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Union.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Union.java
@@ -22,23 +22,37 @@ import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.sql.SqlKind;
 
+import java.util.Collections;
 import java.util.List;
 
 /**
  * Relational expression that returns the union of the rows of its inputs, optionally eliminating
  * duplicates.
  *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ *
  * <p>Corresponds to SQL {@code UNION} and {@code UNION ALL}.
  */
 public abstract class Union extends SetOp {
     // ~ Constructors -----------------------------------------------------------
 
+    protected Union(
+            RelOptCluster cluster,
+            RelTraitSet traits,
+            List<RelHint> hints,
+            List<RelNode> inputs,
+            boolean all) {
+        super(cluster, traits, hints, inputs, SqlKind.UNION, all);
+    }
+
     protected Union(RelOptCluster cluster, RelTraitSet traits, List<RelNode> inputs, boolean all) {
-        super(cluster, traits, inputs, SqlKind.UNION, all);
+        super(cluster, traits, Collections.emptyList(), inputs, SqlKind.UNION, all);
     }
 
     /** Creates a Union by parsing serialized output. */
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Values.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Values.java
index 7aa0d0cafb0..e4a6eba4785 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Values.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Values.java
@@ -26,6 +26,8 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.AbstractRelNode;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -35,15 +37,23 @@ import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.Pair;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
-/** Relational expression whose value is a sequence of zero or more literal row values. */
-public abstract class Values extends AbstractRelNode {
+/**
+ * Relational expression whose value is a sequence of zero or more literal row values.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ */
+public abstract class Values extends AbstractRelNode implements Hintable {
 
     public static final Predicate<? super Values> IS_EMPTY_J = Values::isEmpty;
 
+    protected final com.google.common.collect.ImmutableList<RelHint> hints;
+
     @SuppressWarnings("Guava")
     @Deprecated // to be removed before 2.0
     public static final com.google.common.base.Predicate<? super Values> IS_EMPTY = Values::isEmpty;
@@ -68,12 +78,14 @@ public abstract class Values extends AbstractRelNode {
      * not modify them after this call, otherwise bad things will happen.
      *
      * @param cluster Cluster that this relational expression belongs to
+     * @param hints Hints for this node
      * @param rowType Row type for tuples produced by this rel
      * @param tuples 2-dimensional array of tuple values to be produced; outer list contains tuples;
      *     each inner list is one tuple; all tuples must be of same length, conforming to rowType
      */
     protected Values(
             RelOptCluster cluster,
+            List<RelHint> hints,
             RelDataType rowType,
             com.google.common.collect.ImmutableList<
                             com.google.common.collect.ImmutableList<RexLiteral>>
@@ -82,9 +94,31 @@ public abstract class Values extends AbstractRelNode {
         super(cluster, traits);
         this.rowType = rowType;
         this.tuples = tuples;
+        this.hints = com.google.common.collect.ImmutableList.copyOf(hints);
         assert assertRowType();
     }
 
+    /**
+     * Creates a new Values.
+     *
+     * <p>Note that tuples passed in become owned by this rel (without a deep copy), so caller must
+     * not modify them after this call, otherwise bad things will happen.
+     *
+     * @param cluster Cluster that this relational expression belongs to
+     * @param rowType Row type for tuples produced by this rel
+     * @param tuples 2-dimensional array of tuple values to be produced; outer list contains tuples;
+     *     each inner list is one tuple; all tuples must be of same length, conforming to rowType
+     */
+    protected Values(
+            RelOptCluster cluster,
+            RelDataType rowType,
+            com.google.common.collect.ImmutableList<
+                            com.google.common.collect.ImmutableList<RexLiteral>>
+                    tuples,
+            RelTraitSet traits) {
+        this(cluster, Collections.emptyList(), rowType, tuples, traits);
+    }
+
     /** Creates a Values by parsing serialized output. */
     public Values(RelInput input) {
         this(
@@ -207,4 +241,9 @@ public abstract class Values extends AbstractRelNode {
         }
         return relWriter;
     }
+
+    @Override
+    public com.google.common.collect.ImmutableList<RelHint> getHints() {
+        return hints;
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Window.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Window.java
index 26f4de75f9e..7784dd8364c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Window.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/core/Window.java
@@ -29,6 +29,8 @@ import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexCall;
@@ -46,30 +48,36 @@ import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import java.util.AbstractList;
+import java.util.Collections;
 import java.util.List;
 import java.util.Objects;
 
 /**
  * A relational expression representing a set of window aggregates.
  *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ *
  * <p>A Window can handle several window aggregate functions, over several partitions, with pre- and
  * post-expressions, and an optional post-filter. Each of the partitions is defined by a partition
  * key (zero or more columns) and a range (logical or physical). The partitions expect the data to
  * be sorted correctly on input to the relational expression.
  *
- * <p>Each {@link Group} has a set of {@link org.apache.calcite.rex.RexOver} objects.
+ * <p>Each {@link Window.Group} has a set of {@link org.apache.calcite.rex.RexOver} objects.
  *
  * <p>Created by {@link org.apache.calcite.rel.rules.ProjectToWindowRule}.
  */
-public abstract class Window extends SingleRel {
+public abstract class Window extends SingleRel implements Hintable {
     public final com.google.common.collect.ImmutableList<Group> groups;
     public final com.google.common.collect.ImmutableList<RexLiteral> constants;
+    protected final com.google.common.collect.ImmutableList<RelHint> hints;
 
     /**
      * Creates a window relational expression.
      *
      * @param cluster Cluster
      * @param traitSet Trait set
+     * @param hints Hints for this node
      * @param input Input relational expression
      * @param constants List of constants that are additional inputs
      * @param rowType Output row type
@@ -78,6 +86,7 @@ public abstract class Window extends SingleRel {
     public Window(
             RelOptCluster cluster,
             RelTraitSet traitSet,
+            List<RelHint> hints,
             RelNode input,
             List<RexLiteral> constants,
             RelDataType rowType,
@@ -87,6 +96,27 @@ public abstract class Window extends SingleRel {
         assert rowType != null;
         this.rowType = rowType;
         this.groups = com.google.common.collect.ImmutableList.copyOf(groups);
+        this.hints = com.google.common.collect.ImmutableList.copyOf(hints);
+    }
+
+    /**
+     * Creates a window relational expression.
+     *
+     * @param cluster Cluster
+     * @param traitSet Trait set
+     * @param input Input relational expression
+     * @param constants List of constants that are additional inputs
+     * @param rowType Output row type
+     * @param groups Windows
+     */
+    public Window(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode input,
+            List<RexLiteral> constants,
+            RelDataType rowType,
+            List<Group> groups) {
+        this(cluster, traitSet, Collections.emptyList(), input, constants, rowType, groups);
     }
 
     @Override
@@ -223,8 +253,8 @@ public abstract class Window extends SingleRel {
         private final String digest;
 
         /**
-         * List of {@link RexWinAggCall} objects, each of which is a call to a {@link
-         * SqlAggFunction}.
+         * List of {@link Window.RexWinAggCall} objects, each of which is a call to a {@link
+         * org.apache.calcite.sql.SqlAggFunction}.
          */
         public final com.google.common.collect.ImmutableList<RexWinAggCall> aggCalls;
 
@@ -355,10 +385,11 @@ public abstract class Window extends SingleRel {
     /**
      * A call to a windowed aggregate function.
      *
-     * <p>Belongs to a {@link Group}.
+     * <p>Belongs to a {@link Window.Group}.
      *
-     * <p>It's a bastard son of a {@link RexCall}; similar enough that it gets visited by a {@link
-     * org.apache.calcite.rex.RexVisitor}, but it also has some extra data members.
+     * <p>It's a bastard son of a {@link org.apache.calcite.rex.RexCall}; similar enough that it
+     * gets visited by a {@link org.apache.calcite.rex.RexVisitor}, but it also has some extra data
+     * members.
      */
     public static class RexWinAggCall extends RexCall {
         /** Ordinal of this aggregate within its partition. */
@@ -432,4 +463,9 @@ public abstract class Window extends SingleRel {
             return super.clone(type, operands);
         }
     }
+
+    @Override
+    public com.google.common.collect.ImmutableList<RelHint> getHints() {
+        return hints;
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/HintPredicates.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/HintPredicates.java
index 59b53e3e089..a38ad2e141e 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/HintPredicates.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/HintPredicates.java
@@ -18,7 +18,12 @@
 
 package org.apache.calcite.rel.hint;
 
-/** A collection of hint predicates. */
+/**
+ * A collection of hint predicates.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade *
+ * the latest calcite.
+ */
 public abstract class HintPredicates {
     /**
      * A hint predicate that indicates a hint can only be used to the whole query(no specific
@@ -41,6 +46,13 @@ public abstract class HintPredicates {
     public static final HintPredicate TABLE_SCAN =
             new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.TABLE_SCAN);
 
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.Filter} nodes.
+     */
+    public static final HintPredicate FILTER =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.FILTER);
+
     /**
      * A hint predicate that indicates a hint can only be used to {@link
      * org.apache.calcite.rel.core.Project} nodes.
@@ -48,6 +60,27 @@ public abstract class HintPredicates {
     public static final HintPredicate PROJECT =
             new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.PROJECT);
 
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.Correlate} nodes.
+     */
+    public static final HintPredicate CORRELATE =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.CORRELATE);
+
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.SetOp} nodes.
+     */
+    public static final HintPredicate SETOP =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.SETOP);
+
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.Sort} nodes.
+     */
+    public static final HintPredicate SORT =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.SORT);
+
     /**
      * A hint predicate that indicates a hint can only be used to {@link
      * org.apache.calcite.rel.core.Aggregate} nodes.
@@ -62,6 +95,20 @@ public abstract class HintPredicates {
     public static final HintPredicate CALC =
             new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.CALC);
 
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.Values} nodes.
+     */
+    public static final HintPredicate VALUES =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.VALUES);
+
+    /**
+     * A hint predicate that indicates a hint can only be used to {@link
+     * org.apache.calcite.rel.core.Window} nodes.
+     */
+    public static final HintPredicate WINDOW =
+            new NodeTypeHintPredicate(NodeTypeHintPredicate.NodeType.WINDOW);
+
     /**
      * Returns a composed hint predicate that represents a short-circuiting logical AND of an array
      * of hint predicates {@code hintPredicates}. When evaluating the composed predicate, if a
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/NodeTypeHintPredicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/NodeTypeHintPredicate.java
index 9a5bbff528b..4ed6d117c09 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/NodeTypeHintPredicate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/hint/NodeTypeHintPredicate.java
@@ -21,12 +21,21 @@ package org.apache.calcite.rel.hint;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.Calc;
+import org.apache.calcite.rel.core.Correlate;
+import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.SetOp;
+import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.core.Window;
 
 /**
  * A hint predicate that specifies which kind of relational expression the hint can be applied to.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
  */
 public class NodeTypeHintPredicate implements HintPredicate {
 
@@ -44,14 +53,32 @@ public class NodeTypeHintPredicate implements HintPredicate {
         /** The hint would be propagated to the TableScan nodes. */
         TABLE_SCAN(TableScan.class),
 
+        /** The hint would be propagated to the Filter nodes. */
+        FILTER(Filter.class),
+
         /** The hint would be propagated to the Project nodes. */
         PROJECT(Project.class),
 
+        /** The hint would be propagated to the Correlate nodes. */
+        CORRELATE(Correlate.class),
+
         /** The hint would be propagated to the Aggregate nodes. */
         AGGREGATE(Aggregate.class),
 
         /** The hint would be propagated to the Calc nodes. */
-        CALC(Calc.class);
+        CALC(Calc.class),
+
+        /** The hint would be propagated to the SetOp(Union, Intersect, Minus) nodes. */
+        SETOP(SetOp.class),
+
+        /** The hint would be propagated to the Sort nodes. */
+        SORT(Sort.class),
+
+        /** The hint would be propagated to the Values nodes. */
+        VALUES(Values.class),
+
+        /** The hint would be propagated to the Window nodes. */
+        WINDOW(Window.class);
 
         /** Relational expression clazz that the hint can apply to. */
         private Class<?> relClazz;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
index 01f00b945f9..2c0703c5487 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalCorrelate.java
@@ -28,19 +28,26 @@ import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Correlate;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Litmus;
 
+import java.util.Collections;
+import java.util.List;
+
 /**
  * A relational operator that performs nested-loop joins.
  *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ *
  * <p>It behaves like a kind of {@link org.apache.calcite.rel.core.Join}, but works by setting
  * variables in its environment and restarting its right-hand input.
  *
  * <p>A LogicalCorrelate is used to represent a correlated query. One implementation strategy is to
  * de-correlate the expression.
  *
- * @see CorrelationId
+ * @see org.apache.calcite.rel.core.CorrelationId
  */
 public final class LogicalCorrelate extends Correlate {
     // ~ Instance fields --------------------------------------------------------
@@ -51,6 +58,7 @@ public final class LogicalCorrelate extends Correlate {
      * Creates a LogicalCorrelate.
      *
      * @param cluster cluster this relational expression belongs to
+     * @param hints hints for this node
      * @param left left input relational expression
      * @param right right input relational expression
      * @param correlationId variable name for the row of left input
@@ -60,15 +68,45 @@ public final class LogicalCorrelate extends Correlate {
     public LogicalCorrelate(
             RelOptCluster cluster,
             RelTraitSet traitSet,
+            List<RelHint> hints,
             RelNode left,
             RelNode right,
             CorrelationId correlationId,
             ImmutableBitSet requiredColumns,
             JoinRelType joinType) {
-        super(cluster, traitSet, left, right, correlationId, requiredColumns, joinType);
+        super(cluster, traitSet, hints, left, right, correlationId, requiredColumns, joinType);
         assert !CalciteSystemProperty.DEBUG.value() || isValid(Litmus.THROW, null);
     }
 
+    /**
+     * Creates a LogicalCorrelate.
+     *
+     * @param cluster cluster this relational expression belongs to
+     * @param left left input relational expression
+     * @param right right input relational expression
+     * @param correlationId variable name for the row of left input
+     * @param requiredColumns Required columns
+     * @param joinType join type
+     */
+    public LogicalCorrelate(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode left,
+            RelNode right,
+            CorrelationId correlationId,
+            ImmutableBitSet requiredColumns,
+            JoinRelType joinType) {
+        this(
+                cluster,
+                traitSet,
+                Collections.emptyList(),
+                left,
+                right,
+                correlationId,
+                requiredColumns,
+                joinType);
+    }
+
     /** Creates a LogicalCorrelate by parsing serialized output. */
     public LogicalCorrelate(RelInput input) {
         this(
@@ -81,6 +119,20 @@ public final class LogicalCorrelate extends Correlate {
                 input.getEnum("joinType", JoinRelType.class));
     }
 
+    /** Creates a LogicalCorrelate. */
+    public static LogicalCorrelate create(
+            RelNode left,
+            RelNode right,
+            List<RelHint> hints,
+            CorrelationId correlationId,
+            ImmutableBitSet requiredColumns,
+            JoinRelType joinType) {
+        final RelOptCluster cluster = left.getCluster();
+        final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE);
+        return new LogicalCorrelate(
+                cluster, traitSet, hints, left, right, correlationId, requiredColumns, joinType);
+    }
+
     /** Creates a LogicalCorrelate. */
     public static LogicalCorrelate create(
             RelNode left,
@@ -106,11 +158,31 @@ public final class LogicalCorrelate extends Correlate {
             JoinRelType joinType) {
         assert traitSet.containsIfApplicable(Convention.NONE);
         return new LogicalCorrelate(
-                getCluster(), traitSet, left, right, correlationId, requiredColumns, joinType);
+                getCluster(),
+                traitSet,
+                hints,
+                left,
+                right,
+                correlationId,
+                requiredColumns,
+                joinType);
     }
 
     @Override
     public RelNode accept(RelShuttle shuttle) {
         return shuttle.visit(this);
     }
+
+    @Override
+    public RelNode withHints(List<RelHint> hintList) {
+        return new LogicalCorrelate(
+                getCluster(),
+                traitSet,
+                hintList,
+                left,
+                right,
+                correlationId,
+                requiredColumns,
+                joinType);
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
index 93f3e61bcbd..05852df6926 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
@@ -29,16 +29,24 @@ import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.Litmus;
 
+import java.util.List;
 import java.util.Objects;
 import java.util.Set;
 
-/** Sub-class of {@link Filter} not targeted at any particular engine or calling convention. */
+/**
+ * Sub-class of {@link org.apache.calcite.rel.core.Filter} not targeted at any particular engine or
+ * calling convention.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ */
 public final class LogicalFilter extends Filter {
     private final com.google.common.collect.ImmutableSet<CorrelationId> variablesSet;
 
@@ -50,6 +58,7 @@ public final class LogicalFilter extends Filter {
      * <p>Use {@link #create} unless you know what you're doing.
      *
      * @param cluster Cluster that this relational expression belongs to
+     * @param hints Hints for this node
      * @param child Input relational expression
      * @param condition Boolean expression which determines whether a row is allowed to pass
      * @param variablesSet Correlation variables set by this relational expression to be used by
@@ -58,14 +67,30 @@ public final class LogicalFilter extends Filter {
     public LogicalFilter(
             RelOptCluster cluster,
             RelTraitSet traitSet,
+            List<RelHint> hints,
             RelNode child,
             RexNode condition,
             com.google.common.collect.ImmutableSet<CorrelationId> variablesSet) {
-        super(cluster, traitSet, child, condition);
+        super(cluster, traitSet, hints, child, condition);
         this.variablesSet = Objects.requireNonNull(variablesSet);
         assert isValid(Litmus.THROW, null);
     }
 
+    public LogicalFilter(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode child,
+            RexNode condition,
+            com.google.common.collect.ImmutableSet<CorrelationId> variablesSet) {
+        this(
+                cluster,
+                traitSet,
+                com.google.common.collect.ImmutableList.of(),
+                child,
+                condition,
+                variablesSet);
+    }
+
     @Deprecated // to be removed before 2.0
     public LogicalFilter(
             RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode condition) {
@@ -120,7 +145,7 @@ public final class LogicalFilter extends Filter {
 
     public LogicalFilter copy(RelTraitSet traitSet, RelNode input, RexNode condition) {
         assert traitSet.containsIfApplicable(Convention.NONE);
-        return new LogicalFilter(getCluster(), traitSet, input, condition, variablesSet);
+        return new LogicalFilter(getCluster(), traitSet, hints, input, condition, variablesSet);
     }
 
     @Override
@@ -142,4 +167,9 @@ public final class LogicalFilter extends Filter {
     public int deepHashCode() {
         return Objects.hash(deepHashCode0(), variablesSet);
     }
+
+    @Override
+    public RelNode withHints(List<RelHint> hintList) {
+        return new LogicalFilter(getCluster(), traitSet, hintList, input, condition, variablesSet);
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
index ff5ef05c3e3..b3975937604 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalIntersect.java
@@ -25,7 +25,9 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Intersect;
+import org.apache.calcite.rel.hint.RelHint;
 
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -35,6 +37,20 @@ import java.util.List;
 public final class LogicalIntersect extends Intersect {
     // ~ Constructors -----------------------------------------------------------
 
+    /**
+     * Creates a LogicalIntersect.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     */
+    public LogicalIntersect(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            List<RelHint> hints,
+            List<RelNode> inputs,
+            boolean all) {
+        super(cluster, traitSet, hints, inputs, all);
+    }
+
     /**
      * Creates a LogicalIntersect.
      *
@@ -42,7 +58,7 @@ public final class LogicalIntersect extends Intersect {
      */
     public LogicalIntersect(
             RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-        super(cluster, traitSet, inputs, all);
+        this(cluster, traitSet, Collections.emptyList(), inputs, all);
     }
 
     @Deprecated // to be removed before 2.0
@@ -66,11 +82,16 @@ public final class LogicalIntersect extends Intersect {
 
     @Override
     public LogicalIntersect copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-        return new LogicalIntersect(getCluster(), traitSet, inputs, all);
+        return new LogicalIntersect(getCluster(), traitSet, hints, inputs, all);
     }
 
     @Override
     public RelNode accept(RelShuttle shuttle) {
         return shuttle.visit(this);
     }
+
+    @Override
+    public RelNode withHints(List<RelHint> hintList) {
+        return new LogicalIntersect(getCluster(), traitSet, hintList, inputs, all);
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
index ea2eb044156..cf04c1720da 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalMinus.java
@@ -25,13 +25,35 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Minus;
+import org.apache.calcite.rel.hint.RelHint;
 
+import java.util.Collections;
 import java.util.List;
 
-/** Sub-class of {@link Minus} not targeted at any particular engine or calling convention. */
+/**
+ * Sub-class of {@link org.apache.calcite.rel.core.Minus} not targeted at any particular engine or
+ * calling convention.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ */
 public final class LogicalMinus extends Minus {
     // ~ Constructors -----------------------------------------------------------
 
+    /**
+     * Creates a LogicalMinus.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     */
+    public LogicalMinus(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            List<RelHint> hints,
+            List<RelNode> inputs,
+            boolean all) {
+        super(cluster, traitSet, hints, inputs, all);
+    }
+
     /**
      * Creates a LogicalMinus.
      *
@@ -39,7 +61,7 @@ public final class LogicalMinus extends Minus {
      */
     public LogicalMinus(
             RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-        super(cluster, traitSet, inputs, all);
+        this(cluster, traitSet, Collections.emptyList(), inputs, all);
     }
 
     @Deprecated // to be removed before 2.0
@@ -64,11 +86,16 @@ public final class LogicalMinus extends Minus {
     @Override
     public LogicalMinus copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
         assert traitSet.containsIfApplicable(Convention.NONE);
-        return new LogicalMinus(getCluster(), traitSet, inputs, all);
+        return new LogicalMinus(getCluster(), traitSet, hints, inputs, all);
     }
 
     @Override
     public RelNode accept(RelShuttle shuttle) {
         return shuttle.visit(this);
     }
+
+    @Override
+    public RelNode withHints(List<RelHint> hintList) {
+        return new LogicalMinus(getCluster(), traitSet, hintList, inputs, all);
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalSort.java
index 437744ee039..e0eda26b83a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalSort.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalSort.java
@@ -27,9 +27,19 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rex.RexNode;
 
-/** Sub-class of {@link Sort} not targeted at any particular engine or calling convention. */
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Sub-class of {@link org.apache.calcite.rel.core.Sort} not targeted at any particular engine or
+ * calling convention.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ */
 public final class LogicalSort extends Sort {
     private LogicalSort(
             RelOptCluster cluster,
@@ -38,7 +48,18 @@ public final class LogicalSort extends Sort {
             RelCollation collation,
             RexNode offset,
             RexNode fetch) {
-        super(cluster, traitSet, input, collation, offset, fetch);
+        this(cluster, traitSet, Collections.emptyList(), input, collation, offset, fetch);
+    }
+
+    private LogicalSort(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            List<RelHint> hints,
+            RelNode input,
+            RelCollation collation,
+            RexNode offset,
+            RexNode fetch) {
+        super(cluster, traitSet, hints, input, collation, offset, fetch);
         assert traitSet.containsIfApplicable(Convention.NONE);
     }
 
@@ -72,11 +93,17 @@ public final class LogicalSort extends Sort {
             RelCollation newCollation,
             RexNode offset,
             RexNode fetch) {
-        return new LogicalSort(getCluster(), traitSet, newInput, newCollation, offset, fetch);
+        return new LogicalSort(
+                getCluster(), traitSet, hints, newInput, newCollation, offset, fetch);
     }
 
     @Override
     public RelNode accept(RelShuttle shuttle) {
         return shuttle.visit(this);
     }
+
+    @Override
+    public RelNode withHints(List<RelHint> hintList) {
+        return new LogicalSort(getCluster(), traitSet, hintList, input, collation, offset, fetch);
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
index 25b4420c188..24de518359b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalUnion.java
@@ -25,13 +25,35 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Union;
+import org.apache.calcite.rel.hint.RelHint;
 
+import java.util.Collections;
 import java.util.List;
 
-/** Sub-class of {@link Union} not targeted at any particular engine or calling convention. */
+/**
+ * Sub-class of {@link org.apache.calcite.rel.core.Union} not targeted at any particular engine or
+ * calling convention.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ */
 public final class LogicalUnion extends Union {
     // ~ Constructors -----------------------------------------------------------
 
+    /**
+     * Creates a LogicalUnion.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     */
+    public LogicalUnion(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            List<RelHint> hints,
+            List<RelNode> inputs,
+            boolean all) {
+        super(cluster, traitSet, hints, inputs, all);
+    }
+
     /**
      * Creates a LogicalUnion.
      *
@@ -39,7 +61,7 @@ public final class LogicalUnion extends Union {
      */
     public LogicalUnion(
             RelOptCluster cluster, RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
-        super(cluster, traitSet, inputs, all);
+        this(cluster, traitSet, Collections.emptyList(), inputs, all);
     }
 
     @Deprecated // to be removed before 2.0
@@ -63,11 +85,16 @@ public final class LogicalUnion extends Union {
 
     public LogicalUnion copy(RelTraitSet traitSet, List<RelNode> inputs, boolean all) {
         assert traitSet.containsIfApplicable(Convention.NONE);
-        return new LogicalUnion(getCluster(), traitSet, inputs, all);
+        return new LogicalUnion(getCluster(), traitSet, hints, inputs, all);
     }
 
     @Override
     public RelNode accept(RelShuttle shuttle) {
         return shuttle.visit(this);
     }
+
+    @Override
+    public RelNode withHints(List<RelHint> hintList) {
+        return new LogicalUnion(getCluster(), traitSet, hintList, inputs, all);
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
index c8ff3179c7f..7dd2c6b66c4 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.metadata.RelMdCollation;
 import org.apache.calcite.rel.metadata.RelMdDistribution;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
@@ -35,12 +36,41 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.sql.type.SqlTypeName;
 
 import java.math.BigDecimal;
+import java.util.Collections;
 import java.util.List;
 
-/** Sub-class of {@link Values} not targeted at any particular engine or calling convention. */
+/**
+ * Sub-class of {@link org.apache.calcite.rel.core.Values} not targeted at any particular engine or
+ * calling convention.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ */
 public class LogicalValues extends Values {
     // ~ Constructors -----------------------------------------------------------
 
+    /**
+     * Creates a LogicalValues.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     *
+     * @param cluster Cluster that this relational expression belongs to
+     * @param hints Hints for this node
+     * @param rowType Row type for tuples produced by this rel
+     * @param tuples 2-dimensional array of tuple values to be produced; outer list contains tuples;
+     *     each inner list is one tuple; all tuples must be of same length, conforming to rowType
+     */
+    public LogicalValues(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            List<RelHint> hints,
+            RelDataType rowType,
+            com.google.common.collect.ImmutableList<
+                            com.google.common.collect.ImmutableList<RexLiteral>>
+                    tuples) {
+        super(cluster, hints, rowType, tuples, traitSet);
+    }
+
     /**
      * Creates a LogicalValues.
      *
@@ -58,7 +88,7 @@ public class LogicalValues extends Values {
             com.google.common.collect.ImmutableList<
                             com.google.common.collect.ImmutableList<RexLiteral>>
                     tuples) {
-        super(cluster, rowType, tuples, traitSet);
+        this(cluster, traitSet, Collections.emptyList(), rowType, tuples);
     }
 
     @Deprecated // to be removed before 2.0
@@ -131,4 +161,9 @@ public class LogicalValues extends Values {
     public RelNode accept(RelShuttle shuttle) {
         return shuttle.visit(this);
     }
+
+    @Override
+    public RelNode withHints(List<RelHint> hintList) {
+        return new LogicalValues(getCluster(), traitSet, hintList, rowType, tuples);
+    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
index ef5f87092c0..6c9cc5a1c5e 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
@@ -25,6 +25,7 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Window;
+import org.apache.calcite.rel.hint.RelHint;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
@@ -44,13 +45,20 @@ import org.apache.calcite.util.Util;
 import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.IdentityHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
-/** Sub-class of {@link Window} not targeted at any particular engine or calling convention. */
+/**
+ * Sub-class of {@link org.apache.calcite.rel.core.Window} not targeted at any particular engine or
+ * calling convention.
+ *
+ * <p>Temporarily copy from calcite to cherry-pick [CALCITE-5107] and will be removed when upgrade
+ * the latest calcite.
+ */
 public final class LogicalWindow extends Window {
     /**
      * Creates a LogicalWindow.
@@ -59,6 +67,7 @@ public final class LogicalWindow extends Window {
      *
      * @param cluster Cluster
      * @param traitSet Trait set
+     * @param hints hints for this node
      * @param input Input relational expression
      * @param constants List of constants that are additional inputs
      * @param rowType Output row type
@@ -67,16 +76,33 @@ public final class LogicalWindow extends Window {
     public LogicalWindow(
             RelOptCluster cluster,
             RelTraitSet traitSet,
+            List<RelHint> hints,
             RelNode input,
             List<RexLiteral> constants,
             RelDataType rowType,
             List<Group> groups) {
-        super(cluster, traitSet, input, constants, rowType, groups);
+        super(cluster, traitSet, hints, input, constants, rowType, groups);
+    }
+
+    /**
+     * Creates a LogicalWindow.
+     *
+     * <p>Use {@link #create} unless you know what you're doing.
+     */
+    public LogicalWindow(
+            RelOptCluster cluster,
+            RelTraitSet traitSet,
+            RelNode input,
+            List<RexLiteral> constants,
+            RelDataType rowType,
+            List<Group> groups) {
+        this(cluster, traitSet, Collections.emptyList(), input, constants, rowType, groups);
     }
 
     @Override
     public LogicalWindow copy(RelTraitSet traitSet, List<RelNode> inputs) {
-        return new LogicalWindow(getCluster(), traitSet, sole(inputs), constants, rowType, groups);
+        return new LogicalWindow(
+                getCluster(), traitSet, hints, sole(inputs), constants, rowType, groups);
     }
 
     /**
@@ -147,7 +173,7 @@ public final class LogicalWindow extends Window {
             }
         }
 
-        final Map<RexOver, RexWinAggCall> aggMap = new HashMap<>();
+        final Map<RexOver, Window.RexWinAggCall> aggMap = new HashMap<>();
         List<Group> groups = new ArrayList<>();
         for (Map.Entry<WindowKey, Collection<RexOver>> entry : windowMap.asMap().entrySet()) {
             final WindowKey windowKey = entry.getKey();
@@ -184,7 +210,7 @@ public final class LogicalWindow extends Window {
         // Figure out the type of the inputs to the output program.
         // They are: the inputs to this rel, followed by the outputs of
         // each window.
-        final List<RexWinAggCall> flattenedAggCallList = new ArrayList<>();
+        final List<Window.RexWinAggCall> flattenedAggCallList = new ArrayList<>();
         final List<Map.Entry<String, RelDataType>> fieldList =
                 new ArrayList<>(child.getRowType().getFieldList());
         final int offset = fieldList.size();
@@ -219,7 +245,7 @@ public final class LogicalWindow extends Window {
                 new RexShuttle() {
                     public RexNode visitOver(RexOver over) {
                         // Look up the aggCall which this expr was translated to.
-                        final RexWinAggCall aggCall = aggMap.get(origToNewOver.get(over));
+                        final Window.RexWinAggCall aggCall = aggMap.get(origToNewOver.get(over));
                         assert aggCall != null;
                         assert RelOptUtil.eq(
                                 "over", over.getType(), "aggCall", aggCall.getType(), Litmus.THROW);
@@ -364,4 +390,10 @@ public final class LogicalWindow extends Window {
                         aggWindow.getUpperBound());
         windowMap.put(windowKey, over);
     }
+
+    @Override
+    public RelNode withHints(List<RelHint> hintList) {
+        return new LogicalWindow(
+                getCluster(), traitSet, hintList, input, constants, rowType, groups);
+    }
 }