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


##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.optimize;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.hint.FlinkHints;
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.util.Util;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static java.util.Collections.singletonList;
+
+/**
+ * Resolve and validate Hints, currently only join hints are supported.
+ *
+ * <p>Here the duplicated join hints will not be checked.
+ */
+public class JoinHintResolver extends RelShuttleImpl {
+    private final Set<RelHint> allHints = new HashSet<>();
+    private final Set<RelHint> validHints = new HashSet<>();
+
+    /** Transforms a relational expression into another relational expression. 
*/
+    public List<RelNode> resolve(List<RelNode> roots) {
+        List<RelNode> resolvedRoots =
+                roots.stream().map(node -> 
node.accept(this)).collect(Collectors.toList());
+        validateHints();
+        return resolvedRoots;
+    }
+
+    @Override
+    public RelNode visit(LogicalJoin join) {
+        return visitBiRel(join);
+    }
+
+    private RelNode visitBiRel(BiRel biRel) {
+        Optional<String> leftName = extractAliasOrTableName(biRel.getLeft());
+        Optional<String> rightName = extractAliasOrTableName(biRel.getRight());
+
+        Set<RelHint> existentKVHints = new HashSet<>();
+
+        List<RelHint> newHints =
+                ((Hintable) biRel)
+                        .getHints().stream()
+                                .flatMap(
+                                        h -> {
+                                            if 
(JoinStrategy.isJoinStrategy(h.hintName)) {
+                                                
allHints.add(trimInheritPath(h));
+                                                // if the hint is valid
+                                                List<String> newOptions =
+                                                        getNewJoinHintOptions(
+                                                                leftName,
+                                                                rightName,
+                                                                h.listOptions,
+                                                                h.hintName);
+
+                                                // check whether the join 
hints options are valid
+                                                boolean isValidOption =
+                                                        
JoinStrategy.validOptions(
+                                                                h.hintName, 
newOptions);
+                                                if (isValidOption) {
+                                                    
validHints.add(trimInheritPath(h));
+                                                    // if the hint defines 
more than one args, only
+                                                    // retain the first one
+                                                    return Stream.of(
+                                                            
RelHint.builder(h.hintName)
+                                                                    
.hintOptions(
+                                                                            
singletonList(
+                                                                               
     newOptions.get(
+                                                                               
             0)))
+                                                                    .build());
+                                                } else {
+                                                    // invalid hint
+                                                    return Stream.of();
+                                                }
+                                            } else {
+                                                //                             
                   //
+                                                // filter alias hints

Review Comment:
   nit: remove these unused comments



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.optimize;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.hint.FlinkHints;
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.util.Util;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static java.util.Collections.singletonList;
+
+/**
+ * Resolve and validate Hints, currently only join hints are supported.
+ *
+ * <p>Here the duplicated join hints will not be checked.
+ */
+public class JoinHintResolver extends RelShuttleImpl {
+    private final Set<RelHint> allHints = new HashSet<>();
+    private final Set<RelHint> validHints = new HashSet<>();
+
+    /** Transforms a relational expression into another relational expression. 
*/

Review Comment:
   nit:  should give a more specific description here, e.g., "Resolves and 
validates join hints in the given {@link RelNode} list, will raise an {@link 
ValidationException} for invalid hints..."



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java:
##########
@@ -75,4 +90,88 @@ public static Map<String, String> mergeTableOptions(
         newProps.putAll(hints);
         return Collections.unmodifiableMap(newProps);
     }
+
+    public static Optional<String> getTableAlias(RelNode node) {
+        if (node instanceof Hintable) {
+            Hintable aliasNode = (Hintable) node;
+            List<String> aliasNames =
+                    aliasNode.getHints().stream()
+                            .filter(h -> 
h.hintName.equalsIgnoreCase(FlinkHints.HINT_ALIAS))
+                            .flatMap(h -> h.listOptions.stream())
+                            .collect(Collectors.toList());
+            if (aliasNames.size() > 0) {
+                return Optional.of(aliasNames.get(0));
+            } else {
+                if (canTransposeToTableScan(node)) {
+                    return getTableAlias(node.getInput(0));
+                }
+            }
+        }
+        return Optional.empty();
+    }
+
+    public static boolean canTransposeToTableScan(RelNode node) {
+        // TODO support look up join
+        return node instanceof LogicalProject // computed column on table
+                || node instanceof LogicalFilter;
+    }
+
+    /** Returns the qualified name of a table scan, otherwise returns empty. */
+    public static Optional<String> getTableName(RelOptTable table) {
+        if (table == null) {
+            return Optional.empty();
+        }
+
+        String tableName;
+        if (table instanceof TableSourceTable) {

Review Comment:
   we can use `FlinkPreparingTableBase#getNames` to get table name safely since 
there're two many sub-classes



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.optimize;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.hint.FlinkHints;
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.util.Util;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static java.util.Collections.singletonList;
+
+/**
+ * Resolve and validate Hints, currently only join hints are supported.

Review Comment:
   nit: "Resolve and validate the join hints."



##########
flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.plan.optimize;
+
+import org.apache.flink.table.api.ValidationException;
+import org.apache.flink.table.planner.hint.FlinkHints;
+import org.apache.flink.table.planner.hint.JoinStrategy;
+
+import org.apache.calcite.plan.hep.HepRelVertex;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.BiRel;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.core.TableScan;
+import org.apache.calcite.rel.hint.Hintable;
+import org.apache.calcite.rel.hint.RelHint;
+import org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.calcite.util.Util;
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static java.util.Collections.singletonList;
+
+/**
+ * Resolve and validate Hints, currently only join hints are supported.
+ *
+ * <p>Here the duplicated join hints will not be checked.

Review Comment:
   nit: "Note: duplicate join hints are not checked here."



-- 
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-unsubscr...@flink.apache.org

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

Reply via email to