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

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


The following commit(s) were added to refs/heads/release-1.16 by this push:
     new 357221b9b35 [FLINK-29120][table-planner] Avoid join hint propagating 
into view
357221b9b35 is described below

commit 357221b9b3543447b2b439a413639c0ed201ab35
Author: xuyang <[email protected]>
AuthorDate: Mon Aug 29 15:12:36 2022 +0800

    [FLINK-29120][table-planner] Avoid join hint propagating into view
    
    This closes #20697
    
    (cherry picked from commit 6722c89d0df35643dde38c1b8f096aa785579884)
---
 .../table/planner/calcite/FlinkPlannerImpl.scala   |  29 ++-
 .../planner/plan/hints/batch/JoinHintTestBase.java |  91 ++++++-
 .../plan/hints/batch/BroadcastJoinHintTest.xml     | 289 ++++++++++++++++++++-
 .../plan/hints/batch/NestLoopJoinHintTest.xml      | 289 ++++++++++++++++++++-
 .../plan/hints/batch/ShuffleHashJoinHintTest.xml   | 289 ++++++++++++++++++++-
 .../plan/hints/batch/ShuffleMergeJoinHintTest.xml  | 289 ++++++++++++++++++++-
 .../optimize/ClearQueryBlockAliasResolverTest.xml  | 155 ++++++++++-
 .../planner/plan/optimize/JoinHintResolverTest.xml | 159 +++++++++++-
 8 files changed, 1544 insertions(+), 46 deletions(-)

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 f9488750b53..07969d9e68d 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
@@ -193,17 +193,22 @@ class FlinkPlannerImpl(
   private def rel(validatedSqlNode: SqlNode, sqlValidator: 
FlinkCalciteSqlValidator) = {
     try {
       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()
+      val sqlToRelConverter: SqlToRelConverter = if 
(checkContainJoinHintShuttle.containsJoinHint) {
+        val converter = createSqlToRelConverter(
+          sqlValidator,
+          // disable project merge during sql to rel phase to prevent
+          // incorrect propagation of join hints into child query block
+          sqlToRelConverterConfig.addRelBuilderConfigTransform(c => 
c.withBloat(-1))
+        )
+        // TODO currently, it is a relatively hacked way to tell converter
+        // that this SqlNode tree contains join hints
+        converter.containsJoinHint()
+        converter
+      } else {
+        createSqlToRelConverter(sqlValidator, sqlToRelConverterConfig)
       }
 
       sqlToRelConverter.convertQuery(validatedSqlNode, false, true)
@@ -301,7 +306,7 @@ class FlinkPlannerImpl(
       @Nullable outputType: RelDataType) = {
     try {
       val validatedSqlNode = validateExpression(sqlNode, sqlValidator, 
inputRowType, outputType)
-      val sqlToRelConverter = createSqlToRelConverter(sqlValidator)
+      val sqlToRelConverter = createSqlToRelConverter(sqlValidator, 
sqlToRelConverterConfig)
       val nameToNodeMap = inputRowType.getFieldList.asScala
         .map(field => (field.getName, RexInputRef.of(field.getIndex, 
inputRowType)))
         .toMap[String, RexNode]
@@ -312,14 +317,16 @@ class FlinkPlannerImpl(
     }
   }
 
-  private def createSqlToRelConverter(sqlValidator: SqlValidator): 
SqlToRelConverter = {
+  private def createSqlToRelConverter(
+      sqlValidator: SqlValidator,
+      config: SqlToRelConverter.Config): SqlToRelConverter = {
     new SqlToRelConverter(
       createToRelContext(),
       sqlValidator,
       sqlValidator.getCatalogReader.unwrap(classOf[CalciteCatalogReader]),
       cluster,
       convertletTable,
-      sqlToRelConverterConfig)
+      config)
   }
 
   /** Creates a new instance of [[RelOptTable.ToRelContext]] for 
[[RelOptTable]]. */
diff --git 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java
 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java
index 4af17d5d4fa..c20fe3eb2ff 100644
--- 
a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java
+++ 
b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/hints/batch/JoinHintTestBase.java
@@ -95,6 +95,9 @@ public abstract class JoinHintTestBase extends TableTestBase {
                                 + ")");
 
         util.tableEnv().executeSql("CREATE View V4 as select a3 as a4, b3 as 
b4 from T3");
+
+        util.tableEnv()
+                .executeSql("create view V5 as select T1.* from T1 join T2 on 
T1.a1 = T2.a2");
     }
 
     protected abstract String getTestSingleJoinHint();
@@ -372,31 +375,105 @@ public abstract class JoinHintTestBase extends 
TableTestBase {
     }
 
     @Test
-    public void testJoinHintWithoutAffectingJoinInView() {
+    public void 
testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin() {
+        // the join in V2 will use the planner's default join strategy,
+        // and the join between T1 and V5 will use the tested join hint
+        String sql = "select /*+ %s(T1)*/T1.* from T1 join V5 on T1.a1 = 
V5.a1";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void 
testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin() {
+        // the join in V2 will use the planner's default join strategy,
+        // and the join between T1 and V5 will use the tested join hint
+        String sql = "select /*+ %s(T1)*/* from V5";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void 
testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter() {
         // the join in V2 will use the planner's default join strategy,
-        // and the join between T1 and V2 will use BROADCAST
+        // and the join between T1 and V2 will use the tested join hint
         util.tableEnv()
-                .executeSql("create view V2 as select T1.* from T1 join T2 on 
T1.a1 = T2.a2");
+                .executeSql(
+                        "create view V2 as select T1.* from T1 join T2 on 
T1.a1 = T2.a2 where T1.b1 = 'abc'");
+
+        String sql = "select /*+ %s(T1)*/* from V2";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithSimpleSumInSelectList() {
+        String sql =
+                "select /*+ %s(T1)*/T1.b1, sum(T1.a1) from T1 join T2 on T1.b1 
= T2.b2 group by T1.b1";
 
-        String sql = "select /*+ %s(T1)*/T1.* from T1 join V2 on T1.a1 = 
V2.a1";
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void testJoinHintWithCastInSelectList() {
+        String sql =
+                "select /*+ %s(T1)*/T1.b1, cast(T1.a1 as int) from T1 join T2 
on T1.b1 = T2.b2";
 
         verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
     }
 
     @Test
-    public void testJoinHintWithoutAffectingJoinInSubQuery() {
+    public void 
testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuterJoin() {
         // the join in sub-query will use the planner's default join strategy,
-        // and the join outside will use BROADCAST
+        // and the join outside will use the tested join hint
         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 
testJoinHintWithoutAffectingJoinInSubQueryWhileOuterQueryIsNotJoin() {
+        String sql = "select /*+ %s(T1)*/* from (select T1.* from T1 join T2 
on T1.a1 = T2.a2)";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void 
testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilter() {
+        String sql =
+                "select /*+ %s(T1)*/* from (select T1.* from T1 join T2 on 
T1.a1 = T2.a2 where T1.b1 = 'abc')";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void 
testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlock() {
+        String sql =
+                "select /*+ %s(T1)*/T4.a1, (select count(*) from T1 join T3 on 
T1.a1 = T3.a3) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2 where 
T1.b1 = 'abc') T4";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void 
testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectList()
 {
+        String sql =
+                "select /*+ %s(T1)*/T4.a1, (select count(*) from T1 join 
((select T1.a1 as a3 from T1) union (select a3 from T3)) T3 on T1.a1 = T3.a3 
where T3.a3 = 1) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2) T4";
+
+        verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint()));
+    }
+
+    @Test
+    public void 
testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectFrom()
 {
+        String sql =
+                "select /*+ %s(T1)*/T4.a1 from (select T1.* from T1 join 
((select T1.a1 as a2 from T1) union (select a2 from T2)) T2 on T1.a1 = T2.a2) 
T4";
+
+        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
+        // and the join between T1 and alias V2 will use the tested join hint
         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";
 
diff --git 
a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.xml
 
b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.xml
index 1915709ae29..f93f67a017a 100644
--- 
a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.xml
+++ 
b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/BroadcastJoinHintTest.xml
@@ -434,6 +434,28 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS 
NULL(a1), IS NULL(a2), =(a1
 :- 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="testJoinHintWithCastInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.b1, cast(T1.a1 as int) from T1 
join T2 on T1.b1 = T2.b2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(b1=[$1], EXPR$1=[CAST($0):INTEGER])
++- LogicalJoin(condition=[=($1, $3)], 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=[b1, CAST(a1 AS INTEGER) AS EXPR$1])
++- HashJoin(joinType=[InnerJoin], where=[=(b1, b2)], select=[a1, b1, 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, 
project=[b2], metadata=[]]], fields=[b2])
 ]]>
     </Resource>
   </TestCase>
@@ -838,7 +860,7 @@ NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], 
select=[a1, b1, a2, b2],
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuterJoin">
     <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>
@@ -868,17 +890,198 @@ Calc(select=[a1, b1])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlock">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T4.a1, (select count(*) from T1 join 
T3 on T1.a1 = T3.a3) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2 
where T1.b1 = 'abc') T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+      LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[T4]]]])
+         :- 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=[true], select=[a1, EXPR$0], 
build=[right], singleRowJoin=[true])
+:- Calc(select=[a1])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+:     :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
++- Exchange(distribution=[broadcast])
+   +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0])
+      +- Exchange(distribution=[single])
+         +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+            +- Calc(select=[0 AS $f0])
+               +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], 
select=[a1, a3], build=[right])
+                  :- Exchange(distribution=[hash[a1]])
+                  :  +- TableSourceScan(table=[[default_catalog, 
default_database, T1, project=[a1], metadata=[]]], fields=[a1], hints=[[[ALIAS 
options:[T1]]]])
+                  +- Exchange(distribution=[hash[a3]])
+                     +- TableSourceScan(table=[[default_catalog, 
default_database, T3, project=[a3], metadata=[]]], fields=[a3], hints=[[[ALIAS 
options:[T3]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectFrom">
     <Resource name="sql">
-      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join V2 on T1.a1 = 
V2.a1]]>
+      <![CDATA[select /*+ BROADCAST(T1)*/T4.a1 from (select T1.* from T1 join 
((select T1.a1 as a2 from T1) union (select a2 from T2)) T2 on T1.a1 = T2.a2) 
T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalUnion(all=[false], hints=[[[ALIAS options:[T2]]]])
+         :- LogicalProject(a2=[$0])
+         :  +- LogicalTableScan(table=[[default_catalog, default_database, 
T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a2=[$0])
+            +- LogicalTableScan(table=[[default_catalog, default_database, 
T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, a10], 
build=[right])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, 
project=[a1], metadata=[]]], fields=[a1])
+   +- HashAggregate(isMerge=[true], groupBy=[a1], select=[a1])
+      +- Exchange(distribution=[hash[a1]])
+         +- LocalHashAggregate(groupBy=[a1], select=[a1])
+            +- Union(all=[true], union=[a1])
+               :- 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="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T4.a1, (select count(*) from T1 join 
((select T1.a1 as a3 from T1) union (select a3 from T3)) T3 on T1.a1 = T3.a3 
where T3.a3 = 1) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2) 
T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalFilter(condition=[=($2, 1)])
+      LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+        LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+        LogicalUnion(all=[false])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]])
+      :- 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=[true], select=[a1, $f2], 
build=[right])
+:- 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])
++- Exchange(distribution=[broadcast])
+   +- Calc(select=[*(EXPR$0, EXPR$00) AS $f2])
+      +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[EXPR$0, 
EXPR$00], build=[left], singleRowJoin=[true])
+         :- Exchange(distribution=[broadcast])
+         :  +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS 
EXPR$0])
+         :     +- Exchange(distribution=[single])
+         :        +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+         :           +- Calc(select=[a1], where=[=(a1, 1)])
+         :              +- TableSourceScan(table=[[default_catalog, 
default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1], 
hints=[[[ALIAS options:[T1]]]])
+         +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS 
EXPR$0])
+            +- Exchange(distribution=[single])
+               +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+                  +- HashAggregate(isMerge=[true], groupBy=[a3], select=[a3])
+                     +- Exchange(distribution=[hash[a3]])
+                        +- LocalHashAggregate(groupBy=[a3], select=[a3])
+                           +- Union(all=[true], union=[a3])
+                              :- Calc(select=[CAST(1 AS BIGINT) AS a3], 
where=[=(a1, 1)])
+                              :  +- TableSourceScan(table=[[default_catalog, 
default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1], 
hints=[[[ALIAS options:[T1]]]])
+                              +- Calc(select=[CAST(1 AS BIGINT) AS a3], 
where=[=(a3, 1)])
+                                 +- TableSourceScan(table=[[default_catalog, 
default_database, T3, filter=[], project=[a3], metadata=[]]], fields=[a3], 
hints=[[[ALIAS options:[T3]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from (select T1.* from T1 join T2 
on T1.a1 = T2.a2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[EXPR$0]]]])
+      :- 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, a2)], select=[a1, b1, a2], 
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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from (select T1.* from T1 join T2 
on T1.a1 = T2.a2 where T1.b1 = 'abc')]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[EXPR$0]]]])
+         :- 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, CAST('abc' AS VARCHAR(2147483647)) AS b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join V5 on T1.a1 = 
V5.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]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]])
          :- LogicalTableScan(table=[[default_catalog, default_database, T1]])
          +- LogicalTableScan(table=[[default_catalog, default_database, T2]])
 ]]>
@@ -895,6 +1098,56 @@ Calc(select=[a1, b1])
          :  +- 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="testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from V5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]])
+      :- 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, a2)], select=[a1, b1, a2], 
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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from V2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 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, CAST('abc' AS VARCHAR(2147483647)) AS b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
 ]]>
     </Resource>
   </TestCase>
@@ -1028,6 +1281,32 @@ HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], 
select=[a1, b1], build=[rig
 :  +- 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="testJoinHintWithSimpleSumInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.b1, sum(T1.a1) from T1 join T2 on 
T1.b1 = T2.b2 group by T1.b1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
++- LogicalProject(b1=[$1], a1=[$0])
+   +- LogicalJoin(condition=[=($1, $3)], 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[
+HashAggregate(isMerge=[true], groupBy=[b1], select=[b1, Final_SUM(sum$0) AS 
EXPR$1])
++- Exchange(distribution=[hash[b1]])
+   +- LocalHashAggregate(groupBy=[b1], select=[b1, Partial_SUM(a1) AS sum$0])
+      +- Calc(select=[b1, a1])
+         +- HashJoin(joinType=[InnerJoin], where=[=(b1, b2)], select=[a1, b1, 
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, 
project=[b2], metadata=[]]], fields=[b2])
 ]]>
     </Resource>
   </TestCase>
diff --git 
a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.xml
 
b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.xml
index 36f873fa27f..e7ff453d232 100644
--- 
a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.xml
+++ 
b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/NestLoopJoinHintTest.xml
@@ -433,6 +433,28 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS 
NULL(a1), IS NULL(a2), =(a1
 :- 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="testJoinHintWithCastInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/T1.b1, cast(T1.a1 as int) from T1 
join T2 on T1.b1 = T2.b2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(b1=[$1], EXPR$1=[CAST($0):INTEGER])
++- LogicalJoin(condition=[=($1, $3)], 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=[b1, CAST(a1 AS INTEGER) AS EXPR$1])
++- NestedLoopJoin(joinType=[InnerJoin], where=[=(b1, b2)], select=[a1, b1, 
b2], build=[left])
+   :- Exchange(distribution=[broadcast])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], 
fields=[a1, b1])
+   +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[b2], metadata=[]]], fields=[b2])
 ]]>
     </Resource>
   </TestCase>
@@ -835,7 +857,7 @@ NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], 
select=[a1, b1, a2, b2],
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuterJoin">
     <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>
@@ -865,17 +887,198 @@ Calc(select=[a1, b1])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlock">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/T4.a1, (select count(*) from T1 join 
T3 on T1.a1 = T3.a3) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2 
where T1.b1 = 'abc') T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+      LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[T4]]]])
+         :- 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=[true], select=[a1, EXPR$0], 
build=[right])
+:- Calc(select=[a1])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+:     :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
++- Exchange(distribution=[broadcast])
+   +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0])
+      +- Exchange(distribution=[single])
+         +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+            +- Calc(select=[0 AS $f0])
+               +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], 
select=[a1, a3], build=[right])
+                  :- Exchange(distribution=[hash[a1]])
+                  :  +- TableSourceScan(table=[[default_catalog, 
default_database, T1, project=[a1], metadata=[]]], fields=[a1], hints=[[[ALIAS 
options:[T1]]]])
+                  +- Exchange(distribution=[hash[a3]])
+                     +- TableSourceScan(table=[[default_catalog, 
default_database, T3, project=[a3], metadata=[]]], fields=[a3], hints=[[[ALIAS 
options:[T3]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectFrom">
     <Resource name="sql">
-      <![CDATA[select /*+ NEST_LOOP(T1)*/T1.* from T1 join V2 on T1.a1 = 
V2.a1]]>
+      <![CDATA[select /*+ NEST_LOOP(T1)*/T4.a1 from (select T1.* from T1 join 
((select T1.a1 as a2 from T1) union (select a2 from T2)) T2 on T1.a1 = T2.a2) 
T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalUnion(all=[false], hints=[[[ALIAS options:[T2]]]])
+         :- LogicalProject(a2=[$0])
+         :  +- LogicalTableScan(table=[[default_catalog, default_database, 
T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a2=[$0])
+            +- LogicalTableScan(table=[[default_catalog, default_database, 
T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, a10], 
build=[right])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, 
project=[a1], metadata=[]]], fields=[a1])
+   +- HashAggregate(isMerge=[true], groupBy=[a1], select=[a1])
+      +- Exchange(distribution=[hash[a1]])
+         +- LocalHashAggregate(groupBy=[a1], select=[a1])
+            +- Union(all=[true], union=[a1])
+               :- 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="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/T4.a1, (select count(*) from T1 join 
((select T1.a1 as a3 from T1) union (select a3 from T3)) T3 on T1.a1 = T3.a3 
where T3.a3 = 1) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2) 
T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalFilter(condition=[=($2, 1)])
+      LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+        LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+        LogicalUnion(all=[false])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]])
+      :- 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=[true], select=[a1, $f2], 
build=[right])
+:- 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])
++- Exchange(distribution=[broadcast])
+   +- Calc(select=[*(EXPR$0, EXPR$00) AS $f2])
+      +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[EXPR$0, 
EXPR$00], build=[left], singleRowJoin=[true])
+         :- Exchange(distribution=[broadcast])
+         :  +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS 
EXPR$0])
+         :     +- Exchange(distribution=[single])
+         :        +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+         :           +- Calc(select=[a1], where=[=(a1, 1)])
+         :              +- TableSourceScan(table=[[default_catalog, 
default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1], 
hints=[[[ALIAS options:[T1]]]])
+         +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS 
EXPR$0])
+            +- Exchange(distribution=[single])
+               +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+                  +- HashAggregate(isMerge=[true], groupBy=[a3], select=[a3])
+                     +- Exchange(distribution=[hash[a3]])
+                        +- LocalHashAggregate(groupBy=[a3], select=[a3])
+                           +- Union(all=[true], union=[a3])
+                              :- Calc(select=[CAST(1 AS BIGINT) AS a3], 
where=[=(a1, 1)])
+                              :  +- TableSourceScan(table=[[default_catalog, 
default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1], 
hints=[[[ALIAS options:[T1]]]])
+                              +- Calc(select=[CAST(1 AS BIGINT) AS a3], 
where=[=(a3, 1)])
+                                 +- TableSourceScan(table=[[default_catalog, 
default_database, T3, filter=[], project=[a3], metadata=[]]], fields=[a3], 
hints=[[[ALIAS options:[T3]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/* from (select T1.* from T1 join T2 
on T1.a1 = T2.a2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[EXPR$0]]]])
+      :- 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, a2)], select=[a1, b1, a2], 
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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/* from (select T1.* from T1 join T2 
on T1.a1 = T2.a2 where T1.b1 = 'abc')]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[EXPR$0]]]])
+         :- 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, CAST('abc' AS VARCHAR(2147483647)) AS b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/T1.* from T1 join V5 on T1.a1 = 
V5.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]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]])
          :- LogicalTableScan(table=[[default_catalog, default_database, T1]])
          +- LogicalTableScan(table=[[default_catalog, default_database, T2]])
 ]]>
@@ -892,6 +1095,56 @@ Calc(select=[a1, b1])
          :  +- 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="testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/* from V5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]])
+      :- 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, a2)], select=[a1, b1, a2], 
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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/* from V2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 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, CAST('abc' AS VARCHAR(2147483647)) AS b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
 ]]>
     </Resource>
   </TestCase>
@@ -1024,6 +1277,32 @@ HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], 
select=[a1, b1], build=[rig
 :  +- 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="testJoinHintWithSimpleSumInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ NEST_LOOP(T1)*/T1.b1, sum(T1.a1) from T1 join T2 on 
T1.b1 = T2.b2 group by T1.b1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
++- LogicalProject(b1=[$1], a1=[$0])
+   +- LogicalJoin(condition=[=($1, $3)], 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[
+HashAggregate(isMerge=[true], groupBy=[b1], select=[b1, Final_SUM(sum$0) AS 
EXPR$1])
++- Exchange(distribution=[hash[b1]])
+   +- LocalHashAggregate(groupBy=[b1], select=[b1, Partial_SUM(a1) AS sum$0])
+      +- Calc(select=[b1, a1])
+         +- NestedLoopJoin(joinType=[InnerJoin], where=[=(b1, b2)], 
select=[a1, b1, b2], build=[left])
+            :- Exchange(distribution=[broadcast])
+            :  +- TableSourceScan(table=[[default_catalog, default_database, 
T1]], fields=[a1, b1])
+            +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[b2], metadata=[]]], fields=[b2])
 ]]>
     </Resource>
   </TestCase>
diff --git 
a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.xml
 
b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.xml
index dd016f80566..0d295149aca 100644
--- 
a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.xml
+++ 
b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleHashJoinHintTest.xml
@@ -449,6 +449,29 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS 
NULL(a1), IS NULL(a2), =(a1
 :- 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="testJoinHintWithCastInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/T1.b1, cast(T1.a1 as int) from T1 
join T2 on T1.b1 = T2.b2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(b1=[$1], EXPR$1=[CAST($0):INTEGER])
++- LogicalJoin(condition=[=($1, $3)], 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=[b1, CAST(a1 AS INTEGER) AS EXPR$1])
++- HashJoin(joinType=[InnerJoin], where=[=(b1, b2)], select=[a1, b1, b2], 
build=[left])
+   :- Exchange(distribution=[hash[b1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], 
fields=[a1, b1])
+   +- Exchange(distribution=[hash[b2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[b2], metadata=[]]], fields=[b2])
 ]]>
     </Resource>
   </TestCase>
@@ -867,7 +890,7 @@ NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], 
select=[a1, b1, a2, b2],
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuterJoin">
     <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>
@@ -897,17 +920,198 @@ Calc(select=[a1, b1])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlock">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/T4.a1, (select count(*) from T1 
join T3 on T1.a1 = T3.a3) as cnt from (select T1.* from T1 join T2 on T1.a1 = 
T2.a2 where T1.b1 = 'abc') T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+      LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[T4]]]])
+         :- 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=[true], select=[a1, EXPR$0], 
build=[right], singleRowJoin=[true])
+:- Calc(select=[a1])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+:     :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
++- Exchange(distribution=[broadcast])
+   +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0])
+      +- Exchange(distribution=[single])
+         +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+            +- Calc(select=[0 AS $f0])
+               +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], 
select=[a1, a3], build=[right])
+                  :- Exchange(distribution=[hash[a1]])
+                  :  +- TableSourceScan(table=[[default_catalog, 
default_database, T1, project=[a1], metadata=[]]], fields=[a1], hints=[[[ALIAS 
options:[T1]]]])
+                  +- Exchange(distribution=[hash[a3]])
+                     +- TableSourceScan(table=[[default_catalog, 
default_database, T3, project=[a3], metadata=[]]], fields=[a3], hints=[[[ALIAS 
options:[T3]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectFrom">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/T4.a1 from (select T1.* from T1 
join ((select T1.a1 as a2 from T1) union (select a2 from T2)) T2 on T1.a1 = 
T2.a2) T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalUnion(all=[false], hints=[[[ALIAS options:[T2]]]])
+         :- LogicalProject(a2=[$0])
+         :  +- LogicalTableScan(table=[[default_catalog, default_database, 
T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a2=[$0])
+            +- LogicalTableScan(table=[[default_catalog, default_database, 
T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, a10], 
build=[right])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, 
project=[a1], metadata=[]]], fields=[a1])
+   +- HashAggregate(isMerge=[true], groupBy=[a1], select=[a1])
+      +- Exchange(distribution=[hash[a1]])
+         +- LocalHashAggregate(groupBy=[a1], select=[a1])
+            +- Union(all=[true], union=[a1])
+               :- 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="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/T4.a1, (select count(*) from T1 
join ((select T1.a1 as a3 from T1) union (select a3 from T3)) T3 on T1.a1 = 
T3.a3 where T3.a3 = 1) as cnt from (select T1.* from T1 join T2 on T1.a1 = 
T2.a2) T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalFilter(condition=[=($2, 1)])
+      LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+        LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+        LogicalUnion(all=[false])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]])
+      :- 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=[true], select=[a1, $f2], 
build=[right])
+:- 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])
++- Exchange(distribution=[broadcast])
+   +- Calc(select=[*(EXPR$0, EXPR$00) AS $f2])
+      +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[EXPR$0, 
EXPR$00], build=[left], singleRowJoin=[true])
+         :- Exchange(distribution=[broadcast])
+         :  +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS 
EXPR$0])
+         :     +- Exchange(distribution=[single])
+         :        +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+         :           +- Calc(select=[a1], where=[=(a1, 1)])
+         :              +- TableSourceScan(table=[[default_catalog, 
default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1], 
hints=[[[ALIAS options:[T1]]]])
+         +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS 
EXPR$0])
+            +- Exchange(distribution=[single])
+               +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+                  +- HashAggregate(isMerge=[true], groupBy=[a3], select=[a3])
+                     +- Exchange(distribution=[hash[a3]])
+                        +- LocalHashAggregate(groupBy=[a3], select=[a3])
+                           +- Union(all=[true], union=[a3])
+                              :- Calc(select=[CAST(1 AS BIGINT) AS a3], 
where=[=(a1, 1)])
+                              :  +- TableSourceScan(table=[[default_catalog, 
default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1], 
hints=[[[ALIAS options:[T1]]]])
+                              +- Calc(select=[CAST(1 AS BIGINT) AS a3], 
where=[=(a3, 1)])
+                                 +- TableSourceScan(table=[[default_catalog, 
default_database, T3, filter=[], project=[a3], metadata=[]]], fields=[a3], 
hints=[[[ALIAS options:[T3]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileOuterQueryIsNotJoin">
     <Resource name="sql">
-      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/T1.* from T1 join V2 on T1.a1 = 
V2.a1]]>
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/* from (select T1.* from T1 join 
T2 on T1.a1 = T2.a2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[EXPR$0]]]])
+      :- 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, a2)], select=[a1, b1, a2], 
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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/* from (select T1.* from T1 join 
T2 on T1.a1 = T2.a2 where T1.b1 = 'abc')]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[EXPR$0]]]])
+         :- 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, CAST('abc' AS VARCHAR(2147483647)) AS b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/T1.* from T1 join V5 on T1.a1 = 
V5.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]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]])
          :- LogicalTableScan(table=[[default_catalog, default_database, T1]])
          +- LogicalTableScan(table=[[default_catalog, default_database, T2]])
 ]]>
@@ -924,6 +1128,56 @@ Calc(select=[a1, b1])
          :  +- 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="testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/* from V5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]])
+      :- 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, a2)], select=[a1, b1, a2], 
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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/* from V2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 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, CAST('abc' AS VARCHAR(2147483647)) AS b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
 ]]>
     </Resource>
   </TestCase>
@@ -1059,6 +1313,31 @@ HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], 
select=[a1, b1], build=[rig
 :  +- 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="testJoinHintWithSimpleSumInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_HASH(T1)*/T1.b1, sum(T1.a1) from T1 join T2 
on T1.b1 = T2.b2 group by T1.b1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
++- LogicalProject(b1=[$1], a1=[$0])
+   +- LogicalJoin(condition=[=($1, $3)], 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[
+HashAggregate(isMerge=[false], groupBy=[b1], select=[b1, SUM(a1) AS EXPR$1])
++- Calc(select=[b1, a1])
+   +- HashJoin(joinType=[InnerJoin], where=[=(b1, b2)], select=[a1, b1, b2], 
build=[left])
+      :- Exchange(distribution=[hash[b1]])
+      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], 
fields=[a1, b1])
+      +- Exchange(distribution=[hash[b2]])
+         +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[b2], metadata=[]]], fields=[b2])
 ]]>
     </Resource>
   </TestCase>
diff --git 
a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.xml
 
b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.xml
index e5a1a851edd..c101faa4e50 100644
--- 
a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.xml
+++ 
b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/hints/batch/ShuffleMergeJoinHintTest.xml
@@ -449,6 +449,29 @@ NestedLoopJoin(joinType=[LeftAntiJoin], where=[OR(IS 
NULL(a1), IS NULL(a2), =(a1
 :- 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="testJoinHintWithCastInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/T1.b1, cast(T1.a1 as int) from T1 
join T2 on T1.b1 = T2.b2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(b1=[$1], EXPR$1=[CAST($0):INTEGER])
++- LogicalJoin(condition=[=($1, $3)], 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=[b1, CAST(a1 AS INTEGER) AS EXPR$1])
++- SortMergeJoin(joinType=[InnerJoin], where=[=(b1, b2)], select=[a1, b1, b2])
+   :- Exchange(distribution=[hash[b1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], 
fields=[a1, b1])
+   +- Exchange(distribution=[hash[b2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[b2], metadata=[]]], fields=[b2])
 ]]>
     </Resource>
   </TestCase>
@@ -867,7 +890,7 @@ NestedLoopJoin(joinType=[InnerJoin], where=[>(a1, a2)], 
select=[a1, b1, a2, b2],
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuterJoin">
     <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>
@@ -897,17 +920,198 @@ Calc(select=[a1, b1])
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlock">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/T4.a1, (select count(*) from T1 
join T3 on T1.a1 = T3.a3) as cnt from (select T1.* from T1 join T2 on T1.a1 = 
T2.a2 where T1.b1 = 'abc') T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+      LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[T4]]]])
+         :- 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=[true], select=[a1, EXPR$0], 
build=[right], singleRowJoin=[true])
+:- Calc(select=[a1])
+:  +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+:     :- Exchange(distribution=[hash[a1]])
+:     :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+:     :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+:     +- Exchange(distribution=[hash[a2]])
+:        +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
++- Exchange(distribution=[broadcast])
+   +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS EXPR$0])
+      +- Exchange(distribution=[single])
+         +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+            +- Calc(select=[0 AS $f0])
+               +- HashJoin(joinType=[InnerJoin], where=[=(a1, a3)], 
select=[a1, a3], build=[right])
+                  :- Exchange(distribution=[hash[a1]])
+                  :  +- TableSourceScan(table=[[default_catalog, 
default_database, T1, project=[a1], metadata=[]]], fields=[a1], hints=[[[ALIAS 
options:[T1]]]])
+                  +- Exchange(distribution=[hash[a3]])
+                     +- TableSourceScan(table=[[default_catalog, 
default_database, T3, project=[a3], metadata=[]]], fields=[a3], hints=[[[ALIAS 
options:[T3]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectFrom">
     <Resource name="sql">
-      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/T1.* from T1 join V2 on T1.a1 = 
V2.a1]]>
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/T4.a1 from (select T1.* from T1 
join ((select T1.a1 as a2 from T1) union (select a2 from T2)) T2 on T1.a1 = 
T2.a2) T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]])
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      +- LogicalUnion(all=[false], hints=[[[ALIAS options:[T2]]]])
+         :- LogicalProject(a2=[$0])
+         :  +- LogicalTableScan(table=[[default_catalog, default_database, 
T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+         +- LogicalProject(a2=[$0])
+            +- LogicalTableScan(table=[[default_catalog, default_database, 
T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]])
+]]>
+    </Resource>
+    <Resource name="optimized rel plan">
+      <![CDATA[
+Calc(select=[a1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, a10], 
build=[right])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- TableSourceScan(table=[[default_catalog, default_database, T1, 
project=[a1], metadata=[]]], fields=[a1])
+   +- HashAggregate(isMerge=[true], groupBy=[a1], select=[a1])
+      +- Exchange(distribution=[hash[a1]])
+         +- LocalHashAggregate(groupBy=[a1], select=[a1])
+            +- Union(all=[true], union=[a1])
+               :- 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="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/T4.a1, (select count(*) from T1 
join ((select T1.a1 as a3 from T1) union (select a3 from T3)) T3 on T1.a1 = 
T3.a3 where T3.a3 = 1) as cnt from (select T1.* from T1 join T2 on T1.a1 = 
T2.a2) T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalFilter(condition=[=($2, 1)])
+      LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+        LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+        LogicalUnion(all=[false])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]])
+      :- 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=[true], select=[a1, $f2], 
build=[right])
+:- 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])
++- Exchange(distribution=[broadcast])
+   +- Calc(select=[*(EXPR$0, EXPR$00) AS $f2])
+      +- NestedLoopJoin(joinType=[InnerJoin], where=[true], select=[EXPR$0, 
EXPR$00], build=[left], singleRowJoin=[true])
+         :- Exchange(distribution=[broadcast])
+         :  +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS 
EXPR$0])
+         :     +- Exchange(distribution=[single])
+         :        +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+         :           +- Calc(select=[a1], where=[=(a1, 1)])
+         :              +- TableSourceScan(table=[[default_catalog, 
default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1], 
hints=[[[ALIAS options:[T1]]]])
+         +- HashAggregate(isMerge=[true], select=[Final_COUNT(count1$0) AS 
EXPR$0])
+            +- Exchange(distribution=[single])
+               +- LocalHashAggregate(select=[Partial_COUNT(*) AS count1$0])
+                  +- HashAggregate(isMerge=[true], groupBy=[a3], select=[a3])
+                     +- Exchange(distribution=[hash[a3]])
+                        +- LocalHashAggregate(groupBy=[a3], select=[a3])
+                           +- Union(all=[true], union=[a3])
+                              :- Calc(select=[CAST(1 AS BIGINT) AS a3], 
where=[=(a1, 1)])
+                              :  +- TableSourceScan(table=[[default_catalog, 
default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1], 
hints=[[[ALIAS options:[T1]]]])
+                              +- Calc(select=[CAST(1 AS BIGINT) AS a3], 
where=[=(a3, 1)])
+                                 +- TableSourceScan(table=[[default_catalog, 
default_database, T3, filter=[], project=[a3], metadata=[]]], fields=[a3], 
hints=[[[ALIAS options:[T3]]]])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/* from (select T1.* from T1 join 
T2 on T1.a1 = T2.a2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[EXPR$0]]]])
+      :- 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, a2)], select=[a1, b1, a2], 
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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/* from (select T1.* from T1 join 
T2 on T1.a1 = T2.a2 where T1.b1 = 'abc')]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[EXPR$0]]]])
+         :- 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, CAST('abc' AS VARCHAR(2147483647)) AS b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/T1.* from T1 join V5 on T1.a1 = 
V5.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]]]])
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]])
          :- LogicalTableScan(table=[[default_catalog, default_database, T1]])
          +- LogicalTableScan(table=[[default_catalog, default_database, T2]])
 ]]>
@@ -924,6 +1128,56 @@ Calc(select=[a1, b1])
          :  +- 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="testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/* from V5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]])
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]])
+      :- 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, a2)], select=[a1, b1, a2], 
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])
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/* from V2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1])
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]])
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')])
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 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, CAST('abc' AS VARCHAR(2147483647)) AS b1])
++- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], 
build=[left])
+   :- Exchange(distribution=[hash[a1]])
+   :  +- Calc(select=[a1], where=[=(b1, 'abc')])
+   :     +- TableSourceScan(table=[[default_catalog, default_database, T1, 
filter=[]]], fields=[a1, b1])
+   +- Exchange(distribution=[hash[a2]])
+      +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[a2], metadata=[]]], fields=[a2])
 ]]>
     </Resource>
   </TestCase>
@@ -1059,6 +1313,31 @@ HashJoin(joinType=[LeftSemiJoin], where=[=(a1, a2)], 
select=[a1, b1], build=[rig
 :  +- 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="testJoinHintWithSimpleSumInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ SHUFFLE_MERGE(T1)*/T1.b1, sum(T1.a1) from T1 join T2 
on T1.b1 = T2.b2 group by T1.b1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)])
++- LogicalProject(b1=[$1], a1=[$0])
+   +- LogicalJoin(condition=[=($1, $3)], 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[
+SortAggregate(isMerge=[false], groupBy=[b1], select=[b1, SUM(a1) AS EXPR$1])
++- Calc(select=[b1, a1])
+   +- SortMergeJoin(joinType=[InnerJoin], where=[=(b1, b2)], select=[a1, b1, 
b2])
+      :- Exchange(distribution=[hash[b1]])
+      :  +- TableSourceScan(table=[[default_catalog, default_database, T1]], 
fields=[a1, b1])
+      +- Exchange(distribution=[hash[b2]])
+         +- TableSourceScan(table=[[default_catalog, default_database, T2, 
project=[b2], metadata=[]]], fields=[b2])
 ]]>
     </Resource>
   </TestCase>
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
index b9b59d4215a..7dea9c92062 100644
--- 
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
@@ -237,6 +237,19 @@ 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="testJoinHintWithCastInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.b1, cast(T1.a1 as int) from T1 
join T2 on T1.b1 = T2.b2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(b1=[$1], EXPR$1=[CAST($0):INTEGER]), 
rowType=[RecordType(VARCHAR(2147483647) b1, INTEGER EXPR$1)]
++- LogicalJoin(condition=[=($1, $3)], 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>
@@ -478,7 +491,7 @@ LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), 
rowType=[RecordType(BIGINT a
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuterJoin">
     <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>
@@ -494,9 +507,102 @@ LogicalProject(a1=[$0], b1=[$1]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlock">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T4.a1, (select count(*) from T1 join 
T3 on T1.a1 = T3.a3) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2 
where T1.b1 = 'abc') T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+      LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})]), rowType=[RecordType(BIGINT a1, BIGINT cnt)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, 
VARCHAR(2147483647) b2)]
+      +- 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="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectFrom">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T4.a1 from (select T1.* from T1 join 
((select T1.a1 as a2 from T1) union (select a2 from T2)) T2 on T1.a1 = T2.a2) 
T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0]), rowType=[RecordType(BIGINT a1)]
++- 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)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalUnion(all=[false]), rowType=[RecordType(BIGINT a2)]
+         :- LogicalProject(a2=[$0]), rowType=[RecordType(BIGINT a2)]
+         :  +- LogicalTableScan(table=[[default_catalog, default_database, 
T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a2=[$0]), rowType=[RecordType(BIGINT a2)]
+            +- LogicalTableScan(table=[[default_catalog, default_database, 
T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T4.a1, (select count(*) from T1 join 
((select T1.a1 as a3 from T1) union (select a3 from T3)) T3 on T1.a1 = T3.a3 
where T3.a3 = 1) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2) 
T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalFilter(condition=[=($2, 1)])
+      LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+        LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+        LogicalUnion(all=[false])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})]), rowType=[RecordType(BIGINT a1, BIGINT cnt)]
++- 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="testJoinHintWithoutAffectingJoinInSubQueryWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from (select T1.* from T1 join T2 
on T1.a1 = T2.a2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), 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="testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from (select T1.* from T1 join T2 
on T1.a1 = T2.a2 where T1.b1 = 'abc')]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, 
VARCHAR(2147483647) b2)]
+      +- 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="testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin">
     <Resource name="sql">
-      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join V2 on T1.a1 = 
V2.a1]]>
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join V5 on T1.a1 = 
V5.a1]]>
     </Resource>
     <Resource name="ast">
       <![CDATA[
@@ -507,6 +613,35 @@ LogicalProject(a1=[$0], b1=[$1]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483
       +- 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="testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from V5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), 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="testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from V2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, 
VARCHAR(2147483647) b2)]
+      +- 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>
@@ -589,6 +724,20 @@ 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="testJoinHintWithSimpleSumInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.b1, sum(T1.a1) from T1 join T2 on 
T1.b1 = T2.b2 group by T1.b1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)]), 
rowType=[RecordType(VARCHAR(2147483647) b1, BIGINT EXPR$1)]
++- LogicalProject(b1=[$1], a1=[$0]), rowType=[RecordType(VARCHAR(2147483647) 
b1, BIGINT a1)]
+   +- LogicalJoin(condition=[=($1, $3)], 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>
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
index 0da1883edcd..3368880598e 100644
--- 
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
@@ -237,6 +237,19 @@ 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="testJoinHintWithCastInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.b1, cast(T1.a1 as int) from T1 
join T2 on T1.b1 = T2.b2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(b1=[$1], EXPR$1=[CAST($0):INTEGER]), 
rowType=[RecordType(VARCHAR(2147483647) b1, INTEGER EXPR$1)]
++- LogicalJoin(condition=[=($1, $3)], 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>
@@ -478,7 +491,7 @@ LogicalProject(a1=[$0], b1=[$1], a2=[$2], b2=[$3]), 
rowType=[RecordType(BIGINT a
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInSubQuery">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileArgsCanBeFoundInOuterJoin">
     <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>
@@ -494,17 +507,139 @@ LogicalProject(a1=[$0], b1=[$1]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483
 ]]>
     </Resource>
   </TestCase>
-  <TestCase name="testJoinHintWithoutAffectingJoinInView">
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsSumInQueryBlock">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T4.a1, (select count(*) from T1 join 
T3 on T1.a1 = T3.a3) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2 
where T1.b1 = 'abc') T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+      LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+      LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})]), rowType=[RecordType(BIGINT a1, BIGINT cnt)]
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, 
VARCHAR(2147483647) b2)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[T4]]]]), 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="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectFrom">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T4.a1 from (select T1.* from T1 join 
((select T1.a1 as a2 from T1) union (select a2 from T2)) T2 on T1.a1 = T2.a2) 
T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0]), rowType=[RecordType(BIGINT a1)]
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1, BIGINT a2)]
+      :- LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
+      +- LogicalUnion(all=[false], hints=[[[ALIAS options:[T2]]]]), 
rowType=[RecordType(BIGINT a2)]
+         :- LogicalProject(a2=[$0]), rowType=[RecordType(BIGINT a2)]
+         :  +- LogicalTableScan(table=[[default_catalog, default_database, 
T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+         +- LogicalProject(a2=[$0]), rowType=[RecordType(BIGINT a2)]
+            +- LogicalTableScan(table=[[default_catalog, default_database, 
T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), 
rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)]
+]]>
+    </Resource>
+  </TestCase>
+  <TestCase 
name="testJoinHintWithoutAffectingJoinInSubQueryWhileContainsUnionAndJoinInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T4.a1, (select count(*) from T1 join 
((select T1.a1 as a3 from T1) union (select a3 from T3)) T3 on T1.a1 = T3.a3 
where T3.a3 = 1) as cnt from (select T1.* from T1 join T2 on T1.a1 = T2.a2) 
T4]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], cnt=[$SCALAR_QUERY({
+LogicalAggregate(group=[{}], EXPR$0=[COUNT()])
+  LogicalProject($f0=[0])
+    LogicalFilter(condition=[=($2, 1)])
+      LogicalJoin(condition=[=($0, $2)], joinType=[inner])
+        LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+        LogicalUnion(all=[false])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T1]], 
hints=[[[ALIAS inheritPath:[] options:[T1]]]])
+          LogicalProject(a3=[$0])
+            LogicalTableScan(table=[[default_catalog, default_database, T3]], 
hints=[[[ALIAS inheritPath:[] options:[T3]]]])
+})]), rowType=[RecordType(BIGINT a1, BIGINT cnt)]
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[T4]]]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[T4]]]]), 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="testJoinHintWithoutAffectingJoinInSubQueryWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from (select T1.* from T1 join T2 
on T1.a1 = T2.a2)]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[EXPR$0]]]]), 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="testJoinHintWithoutAffectingJoinInSubQueryWhileRootOfSubQueryIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from (select T1.* from T1 join T2 
on T1.a1 = T2.a2 where T1.b1 = 'abc')]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[EXPR$0]]]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, 
VARCHAR(2147483647) b2)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 0] options:[EXPR$0]]]]), 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="testJoinHintWithoutAffectingJoinInViewWhileArgsCanBeFoundInOuterJoin">
     <Resource name="sql">
-      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join V2 on T1.a1 = 
V2.a1]]>
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.* from T1 join V5 on T1.a1 = 
V5.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)]
+   +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]]), 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="testJoinHintWithoutAffectingJoinInViewWhileOuterQueryIsNotJoin">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from V5]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V5]]]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0] options:[V5]]]]), 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="testJoinHintWithoutAffectingJoinInViewWhileRootOfViewIsFilter">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/* from V2]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, 
VARCHAR(2147483647) b1)]
++- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)]
+   +- LogicalFilter(condition=[=($1, _UTF-16LE'abc')]), 
rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, 
VARCHAR(2147483647) b2)]
+      +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], hints=[[[ALIAS 
inheritPath:[0, 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)]
 ]]>
@@ -589,6 +724,20 @@ 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="testJoinHintWithSimpleSumInSelectList">
+    <Resource name="sql">
+      <![CDATA[select /*+ BROADCAST(T1)*/T1.b1, sum(T1.a1) from T1 join T2 on 
T1.b1 = T2.b2 group by T1.b1]]>
+    </Resource>
+    <Resource name="ast">
+      <![CDATA[
+LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)]), 
rowType=[RecordType(VARCHAR(2147483647) b1, BIGINT EXPR$1)]
++- LogicalProject(b1=[$1], a1=[$0]), rowType=[RecordType(VARCHAR(2147483647) 
b1, BIGINT a1)]
+   +- LogicalJoin(condition=[=($1, $3)], 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>

Reply via email to