You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/12/22 13:40:24 UTC

[GitHub] [flink] lincoln-lil commented on a diff in pull request #21545: [FLINK-30396][table]make alias hint take effect in correlate

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


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttle.java:
##########
@@ -49,14 +52,23 @@ public class ClearJoinHintWithInvalidPropagationShuttle extends RelShuttleImpl {
 
     @Override
     public RelNode visit(LogicalJoin join) {
-        List<RelHint> hints = join.getHints();
+        return visitBiRel(join);
+    }
+
+    @Override
+    public RelNode visit(LogicalCorrelate correlate) {
+        return visitBiRel(correlate);
+    }
+
+    public RelNode visitBiRel(BiRel biRel) {

Review Comment:
   this can be private



##########
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:
   the fixedSizeListOptionChecker is not applicable to CORRELATE  (actually the lookup join which only accepts kvOptions)



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java:
##########
@@ -189,18 +192,33 @@ public static RelNode capitalizeJoinHints(RelNode root) {
 
     private static class CapitalizeJoinHintShuttle extends RelShuttleImpl {
 
+        @Override
+        public RelNode visit(LogicalCorrelate correlate) {
+            return visitBiRel(correlate);
+        }
+
         @Override
         public RelNode visit(LogicalJoin join) {
-            List<RelHint> hints = join.getHints();
+            return visitBiRel(join);
+        }
+
+        public RelNode visitBiRel(BiRel biRel) {

Review Comment:
   can be private



##########
flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearLookupCorrelateHintWithInvalidPropagationShuttleTest.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.alias;
+
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable;
+import org.apache.flink.table.planner.hint.FlinkHints;
+import org.apache.flink.table.planner.plan.nodes.exec.spec.LookupJoinHintTestUtil;
+import org.apache.flink.table.planner.utils.TableTestUtil;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.CorrelationId;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+
+/** Tests clearing lookup join hint with invalid propagation in stream. */
+public class ClearLookupCorrelateHintWithInvalidPropagationShuttleTest

Review Comment:
   It's better to fix the current ClearLookupJoinHintWithInvalidPropagationShuttleTest instead of add another 'LookupCorrelate' test since the lookup join is always converted from a LogicalCorrelate, WDYT?



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java:
##########
@@ -189,18 +192,33 @@ public static RelNode capitalizeJoinHints(RelNode root) {
 
     private static class CapitalizeJoinHintShuttle extends RelShuttleImpl {
 
+        @Override
+        public RelNode visit(LogicalCorrelate correlate) {
+            return visitBiRel(correlate);
+        }
+
         @Override
         public RelNode visit(LogicalJoin join) {
-            List<RelHint> hints = join.getHints();
+            return visitBiRel(join);
+        }
+
+        public RelNode visitBiRel(BiRel biRel) {
+            Hintable hBiRel = (Hintable) biRel;
             AtomicBoolean changed = new AtomicBoolean(false);
             List<RelHint> hintsWithCapitalJoinHints =
-                    hints.stream()
+                    hBiRel.getHints().stream()
                             .map(
                                     hint -> {
                                         String capitalHintName =
                                                 hint.hintName.toUpperCase(Locale.ROOT);
                                         if (JoinStrategy.isJoinStrategy(capitalHintName)) {
                                             changed.set(true);
+                                            if (hint.listOptions.isEmpty()) {

Review Comment:
   We'd better to distinguish the kvOption and listOption via different hint but not the option itself, here we can use `isLookupHint` to determine



-- 
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: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org