lincoln-lil commented on code in PR #21545:
URL: https://github.com/apache/flink/pull/21545#discussion_r1055969354


##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearLookupJoinHintWithInvalidPropagationShuttleTest.java:
##########
@@ -120,22 +125,53 @@ public void 
testClearLookupHintWithInvalidPropagationToViewWhileViewHasLookupHin
     public void testClearLookupHintWithInvalidPropagationToSubQuery() {
         // SELECT /*+ LOOKUP('table'='lookup', 'retry-predicate'='lookup_miss',
         // 'retry-strategy'='fixed_delay', 'fixed-delay'='155 ms', 
'max-attempts'='10',
-        // 'async'='true', 'output-mode'='allow_unordered','capacity'='1000', 
'time-out'='300 s' */*
-        //   FROM (
-        //     SELECT src.a
-        //     FROM src
-        //     JOIN lookup FOR SYSTEM_TIME AS OF T.proctime AS D
-        //       ON T.a = D.id
-        //   ) t1 JOIN src t2 ON t1.a = t2.a
+        // 'async'='true', 'output-mode'='allow_unordered','capacity'='1000', 
'time-out'='300 s')
+        // */ t1.a
+        //  FROM (
+        //      SELECT s.a
+        //      FROM src s
+        //      JOIN lookup FOR SYSTEM_TIME AS OF s.pts AS d
+        //      ON s.a=d.a
+        //  ) t1
+        //  JOIN src t2
+        //  ON t1.a=t2.a
+
+        CorrelationId cid = builder.getCluster().createCorrel();
+        RelDataType aType = builder
+                .getTypeFactory()
+                .createStructType(
+                        
Arrays.asList(builder.getTypeFactory().createSqlType(SqlTypeName.BIGINT)),
+                        Arrays.asList("a")

Review Comment:
   nit: 'Arrays.asList' on single element can be replaced with 
'Collections.singletonList'



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java:
##########
@@ -62,8 +62,10 @@ public static HintStrategyTable createHintStrategyTable() {
                 // internal join hint used for alias
                 .hintStrategy(
                         FlinkHints.HINT_ALIAS,
-                        // currently, only join hints care about query block 
alias
-                        HintStrategy.builder(HintPredicates.JOIN)
+                        // currently, only correlate&join hints care about 
query block alias
+                        HintStrategy.builder(
+                                        HintPredicates.or(
+                                                HintPredicates.CORRELATE, 
HintPredicates.JOIN))
                                 .optionChecker(fixedSizeListOptionChecker(1))

Review Comment:
   sorry, my misunderstood here, this optionChecker is ok.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to