From f794d72903c9c76e89731273dbb60bbee75e3166 Mon Sep 17 00:00:00 2001 From: xuyang Date: Tue, 26 Jul 2022 14:47:27 +0800 Subject: [PATCH] [FLINK-28682][table-planner] Support join hint in batch rules This closes #20359 --- .../calcite/sql2rel/RelDecorrelator.java | 11 + .../calcite/sql2rel/SqlToRelConverter.java | 65 +- ...JoinHintWithInvalidPropagationShuttle.java | 179 +++ .../calcite/RelTimeIndicatorConverter.java | 3 +- .../planner/hint/FlinkHintStrategies.java | 54 + .../flink/table/planner/hint/FlinkHints.java | 101 ++ .../table/planner/hint/JoinStrategy.java | 87 ++ .../ClearQueryBlockAliasResolver.java | 71 + .../plan/optimize/JoinHintResolver.java | 237 ++++ ...reamNonDeterministicUpdatePlanVisitor.java | 9 +- .../calcite/FlinkLogicalRelFactories.scala | 2 +- .../planner/calcite/FlinkPlannerImpl.scala | 49 +- .../planner/plan/nodes/calcite/Sink.scala | 2 +- .../plan/nodes/logical/FlinkLogicalJoin.scala | 21 +- .../BatchCommonSubGraphBasedOptimizer.scala | 15 +- .../SplitPythonConditionFromJoinRule.scala | 1 + ...TemporalJoinRewriteWithUniqueKeyRule.scala | 7 +- .../batch/BatchPhysicalHashJoinRule.scala | 111 +- .../batch/BatchPhysicalJoinRuleBase.scala | 258 +++- .../BatchPhysicalNestedLoopJoinRule.scala | 53 +- .../BatchPhysicalSingleRowJoinRule.scala | 23 +- .../BatchPhysicalSortMergeJoinRule.scala | 12 +- .../planner/plan/utils/FlinkRelOptUtil.scala | 10 +- .../plan/utils/RelTreeWriterImpl.scala | 36 +- ...HintWithInvalidPropagationShuttleTest.java | 225 +++ .../join/joinhint/BroadcastJoinHintTest.java | 35 + .../sql/join/joinhint/JoinHintTestBase.java | 726 ++++++++++ .../join/joinhint/NestLoopJoinHintTest.java | 35 + .../joinhint/ShuffleHashJoinHintTest.java | 35 + .../joinhint/ShuffleMergeJoinHintTest.java | 35 + .../ClearQueryBlockAliasResolverTest.java | 104 ++ .../plan/optimize/JoinHintResolverTest.java | 101 ++ ...nHintWithInvalidPropagationShuttleTest.xml | 150 ++ .../batch/sql/join/ShuffledHashJoinTest.xml | 32 + .../join/joinhint/BroadcastJoinHintTest.xml | 1210 ++++++++++++++++ .../join/joinhint/NestLoopJoinHintTest.xml | 1206 ++++++++++++++++ .../join/joinhint/ShuffleHashJoinHintTest.xml | 1246 +++++++++++++++++ .../joinhint/ShuffleMergeJoinHintTest.xml | 1246 +++++++++++++++++ .../ClearQueryBlockAliasResolverTest.xml | 702 ++++++++++ .../plan/optimize/JoinHintResolverTest.xml | 702 ++++++++++ .../batch/sql/join/ShuffledHashJoinTest.scala | 8 - .../logical/subquery/SubQueryTestBase.scala | 4 +- .../table/planner/utils/TableTestBase.scala | 72 +- 43 files changed, 9123 insertions(+), 168 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttle.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/JoinStrategy.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolver.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/JoinHintTestBase.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.xml create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.xml create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.xml create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.xml create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.xml create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.xml diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java index 07893cc4ad7de..0d13204127890 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java @@ -16,6 +16,8 @@ */ package org.apache.calcite.sql2rel; +import org.apache.flink.table.planner.alias.ClearJoinHintWithInvalidPropagationShuttle; + import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -203,6 +205,15 @@ public static RelNode decorrelateQuery(RelNode rootRel, RelBuilder relBuilder) { // Re-propagate the hints. newRootRel = RelOptUtil.propagateRelHints(newRootRel, true); + // ----- FLINK MODIFICATION BEGIN ----- + + // clear join hints which are propagated into wrong query block + // The hint QueryBlockAlias will be added when building a RelNode tree before. It is used to + // distinguish the query block in the SQL. + newRootRel = newRootRel.accept(new ClearJoinHintWithInvalidPropagationShuttle()); + + // ----- FLINK MODIFICATION END ----- + return newRootRel; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java index 11414792b7402..41b1f266472c6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java @@ -18,8 +18,8 @@ package org.apache.calcite.sql2rel; -import org.apache.flink.table.planner.alias.SubQueryAliasNodeClearShuttle; -import org.apache.flink.table.planner.plan.nodes.calcite.LogicalSubQueryAlias; +import org.apache.flink.table.planner.alias.ClearJoinHintWithInvalidPropagationShuttle; +import org.apache.flink.table.planner.hint.FlinkHints; import org.apache.calcite.avatica.util.Spaces; import org.apache.calcite.linq4j.Ord; @@ -598,8 +598,18 @@ public RelRoot convertQuery(SqlNode query, final boolean needsValidation, final hints = SqlUtil.getRelHint(hintStrategies, select.getHints()); } } - // propagate the hints. + result = RelOptUtil.propagateRelHints(result, false); + + // ----- FLINK MODIFICATION BEGIN ----- + + // clear join hints which are propagated into wrong query block + // The hint QueryBlockAlias will be added when building a RelNode tree before. It is used to + // distinguish the query block in the SQL. + result = result.accept(new ClearJoinHintWithInvalidPropagationShuttle()); + + // ----- FLINK MODIFICATION END ----- + return RelRoot.of(result, validatedRowType, query.getKind()) .withCollation(collation) .withHints(hints); @@ -2025,6 +2035,25 @@ protected void convertFrom(Blackboard bb, SqlNode from) { convertFrom(bb, from, Collections.emptyList()); } + // ----- FLINK MODIFICATION BEGIN ----- + + private boolean containsJoinHint = false; + + /** + * To tell this converter that this SqlNode tree contains join hint and then a query block alias + * will be attached to the root node of the query block. + * + *

The `containsJoinHint` is false default to be compatible with previous behavior and then + * planner can reuse some node. + * + *

TODO At present, it is a relatively hacked way + */ + public void containsJoinHint() { + containsJoinHint = true; + } + + // ----- FLINK MODIFICATION END ----- + /** * Converts a FROM clause into a relational expression. * @@ -2061,6 +2090,36 @@ protected void convertFrom(Blackboard bb, SqlNode from, List fieldNames) } } convertFrom(bb, firstOperand, fieldNameList); + + // ----- FLINK MODIFICATION BEGIN ----- + + // Add a query-block alias hint to distinguish different query levels + // Due to Calcite will expand the whole SQL Rel Node tree that contains query block, + // but sometimes the query block should be perceived such as join hint propagation. + // TODO add query-block alias hint in SqlNode instead of here + if (containsJoinHint) { + RelNode root = bb.root; + + if (root instanceof Hintable) { + RelHint queryBlockAliasHint = + RelHint.builder(FlinkHints.HINT_ALIAS) + .hintOption(call.operand(1).toString()) + .build(); + RelNode newRoot = + ((Hintable) root) + .attachHints( + Collections.singletonList(queryBlockAliasHint)); + boolean isLeaf = leaves.containsKey(root); + if (isLeaf) { + // remove old root node + leaves.remove(root); + } + + bb.setRoot(newRoot, isLeaf); + } + } + + // ----- FLINK MODIFICATION END ----- return; case MATCH_RECOGNIZE: diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttle.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttle.java new file mode 100644 index 0000000000000..426ea68fee2e5 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttle.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.alias; + +import org.apache.flink.table.planner.hint.FlinkHints; +import org.apache.flink.table.planner.hint.JoinStrategy; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelShuttleImpl; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.logical.LogicalJoin; + +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Deque; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Due to Calcite will expand the whole SQL RelNode tree that contains query block, join hints will + * be propagated from root to leaves in the whole RelNode tree. This shuttle is used to clear the + * join hints that are propagated into the query block incorrectly. + * + *

See more at {@see org.apache.calcite.sql2rel.SqlToRelConverter#convertFrom()}. + * + *

TODO some node will be attached join hints when parse SqlNode to RelNode such as Project and + * etc. The join hints on these node can also be cleared. + */ +public class ClearJoinHintWithInvalidPropagationShuttle extends RelShuttleImpl { + + @Override + public RelNode visit(LogicalJoin join) { + List hints = join.getHints(); + + Set allHintNames = + hints.stream().map(hint -> hint.hintName).collect(Collectors.toSet()); + + // there are no join hints on this Join node + if (allHintNames.stream().noneMatch(JoinStrategy::isJoinStrategy)) { + return super.visit(join); + } + + Optional firstAliasHint = + hints.stream() + .filter(hint -> FlinkHints.HINT_ALIAS.equals(hint.hintName)) + .findFirst(); + + // there are no alias hints on this Join node + if (!firstAliasHint.isPresent()) { + return super.visit(join); + } + + List joinHintsFromOuterQueryBlock = + hints.stream() + .filter( + hint -> + JoinStrategy.isJoinStrategy(hint.hintName) + // if the size of inheritPath is bigger than 0, it + // means that this join hint is propagated from its + // parent + && hint.inheritPath.size() + > firstAliasHint.get().inheritPath.size()) + .collect(Collectors.toList()); + + if (joinHintsFromOuterQueryBlock.isEmpty()) { + return super.visit(join); + } + + RelNode newJoin = join; + ClearOuterJoinHintShuttle clearOuterJoinHintShuttle; + + for (RelHint outerJoinHint : joinHintsFromOuterQueryBlock) { + clearOuterJoinHintShuttle = new ClearOuterJoinHintShuttle(outerJoinHint); + newJoin = newJoin.accept(clearOuterJoinHintShuttle); + } + + return super.visit(newJoin); + } + + /** + * A shuttle to clean the join hints which are in outer query block and should not affect the + * query-block inside. + */ + private static class ClearOuterJoinHintShuttle extends RelShuttleImpl { + // the current inheritPath about the join hint that need be removed + private final Deque currentInheritPath; + + // the join hint that need be removed + private final RelHint joinHintNeedRemove; + + public ClearOuterJoinHintShuttle(RelHint joinHintNeedRemove) { + this.joinHintNeedRemove = joinHintNeedRemove; + this.currentInheritPath = new ArrayDeque<>(); + this.currentInheritPath.addAll(joinHintNeedRemove.inheritPath); + } + + @Override + protected RelNode visitChild(RelNode parent, int i, RelNode child) { + currentInheritPath.addLast(i); + RelNode newNode = super.visitChild(parent, i, child); + currentInheritPath.removeLast(); + return newNode; + } + + @Override + public RelNode visit(LogicalJoin join) { + List hints = new ArrayList<>(join.getHints()); + Optional invalidJoinHint = getInvalidJoinHint(hints); + + // if this join node contains the join hint that needs to be removed + if (invalidJoinHint.isPresent()) { + hints.remove(invalidJoinHint.get()); + return super.visit(join.withHints(hints)); + } + + return super.visit(join); + } + + /** + * Get the invalid join hint in this node. + * + *

The invalid join meets the following requirement: + * + *

1. This hint name is same with the join hint that needs to be removed + * + *

2.The length of this hint should be same with the length of propagating this removed + * join hint. + * + *

3. The inherited path of this hint should match the inherited path of this removed + * join hint. + * + * @param hints all hints + * @return return the invalid join hint if exists, else return empty + */ + private Optional getInvalidJoinHint(List hints) { + for (RelHint hint : hints) { + if (hint.hintName.equals(joinHintNeedRemove.hintName) + && isMatchInvalidInheritPath( + new ArrayList<>(currentInheritPath), hint.inheritPath)) { + return Optional.of(hint); + } + } + return Optional.empty(); + } + + private boolean isMatchInvalidInheritPath( + List invalidInheritPath, List checkedInheritPath) { + if (invalidInheritPath.size() != checkedInheritPath.size()) { + return false; + } + + for (int i = 0; i < invalidInheritPath.size(); i++) { + if (!Objects.equals(invalidInheritPath.get(i), checkedInheritPath.get(i))) { + return false; + } + } + return true; + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java index d5cbb555be3e5..5da3c39c4b8da 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/RelTimeIndicatorConverter.java @@ -307,7 +307,8 @@ public RexNode visitInputRef(RexInputRef inputRef) { } } }); - return FlinkLogicalJoin.create(newLeft, newRight, newCondition, join.getJoinType()); + return FlinkLogicalJoin.create( + newLeft, newRight, newCondition, join.getHints(), join.getJoinType()); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java index 6d8e438d47e6c..c3fea35533637 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHintStrategies.java @@ -20,11 +20,14 @@ import org.apache.flink.table.planner.plan.rules.logical.WrapJsonAggFunctionArgumentsRule; +import org.apache.calcite.rel.hint.HintOptionChecker; import org.apache.calcite.rel.hint.HintPredicates; import org.apache.calcite.rel.hint.HintStrategy; import org.apache.calcite.rel.hint.HintStrategyTable; import org.apache.calcite.util.Litmus; +import java.util.Collections; + /** A collection of Flink style {@link HintStrategy}s. */ public abstract class FlinkHintStrategies { @@ -51,6 +54,57 @@ public static HintStrategyTable createHintStrategyTable() { HintStrategy.builder(HintPredicates.AGGREGATE) .excludedRules(WrapJsonAggFunctionArgumentsRule.INSTANCE) .build()) + // internal join hint used for alias + .hintStrategy( + FlinkHints.HINT_ALIAS, + // currently, only join hints care about query block alias + HintStrategy.builder(HintPredicates.JOIN) + .optionChecker(fixedSizeListOptionChecker(1)) + .build()) + // TODO semi/anti join with CORRELATE is not supported + .hintStrategy( + JoinStrategy.BROADCAST.getJoinHintName(), + HintStrategy.builder(HintPredicates.JOIN) + .optionChecker(NON_EMPTY_LIST_OPTION_CHECKER) + .build()) + .hintStrategy( + JoinStrategy.SHUFFLE_HASH.getJoinHintName(), + HintStrategy.builder(HintPredicates.JOIN) + .optionChecker(NON_EMPTY_LIST_OPTION_CHECKER) + .build()) + .hintStrategy( + JoinStrategy.SHUFFLE_MERGE.getJoinHintName(), + HintStrategy.builder(HintPredicates.JOIN) + .optionChecker(NON_EMPTY_LIST_OPTION_CHECKER) + .build()) + .hintStrategy( + JoinStrategy.NEST_LOOP.getJoinHintName(), + HintStrategy.builder(HintPredicates.JOIN) + .optionChecker(NON_EMPTY_LIST_OPTION_CHECKER) + .build()) .build(); } + + private static HintOptionChecker fixedSizeListOptionChecker(int size) { + return (hint, errorHandler) -> + errorHandler.check( + hint.listOptions.size() == size, + "Invalid hint: {}, expecting {} table or view {} " + + "specified in hint {}.", + FlinkHints.stringifyHints(Collections.singletonList(hint)), + size, + size > 1 ? "names" : "name", + hint.hintName); + } + + // ~ hint option checker ------------------------------------------------------------ + + private static final HintOptionChecker NON_EMPTY_LIST_OPTION_CHECKER = + (hint, errorHandler) -> + errorHandler.check( + hint.listOptions.size() > 0, + "Invalid hint: {}, expecting at least " + + "one table or view specified in hint {}.", + FlinkHints.stringifyHints(Collections.singletonList(hint)), + hint.hintName); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java index 8117e4ae98ab8..20f0ee2a03798 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/FlinkHints.java @@ -18,14 +18,24 @@ package org.apache.flink.table.planner.hint; +import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.rules.logical.WrapJsonAggFunctionArgumentsRule; +import org.apache.flink.table.planner.plan.schema.FlinkPreparingTableBase; +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.hint.Hintable; import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.logical.LogicalFilter; +import org.apache.calcite.rel.logical.LogicalProject; +import org.apache.commons.lang3.StringUtils; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; /** Utility class for Flink hints. */ public abstract class FlinkHints { @@ -33,6 +43,9 @@ public abstract class FlinkHints { public static final String HINT_NAME_OPTIONS = "OPTIONS"; + // ~ Internal alias tag hint + public static final String HINT_ALIAS = "ALIAS"; + /** * Internal hint that JSON aggregation function arguments have been wrapped already. See {@link * WrapJsonAggFunctionArgumentsRule}. @@ -75,4 +88,92 @@ public static Map mergeTableOptions( newProps.putAll(hints); return Collections.unmodifiableMap(newProps); } + + public static Optional getTableAlias(RelNode node) { + if (node instanceof Hintable) { + Hintable aliasNode = (Hintable) node; + List 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 getTableName(RelOptTable table) { + if (table == null) { + return Optional.empty(); + } + + String tableName; + if (table instanceof FlinkPreparingTableBase) { + tableName = StringUtils.join(((FlinkPreparingTableBase) table).getNames(), '.'); + } else { + throw new TableException( + String.format( + "Could not get the table name with the unknown table class `%s`", + table.getClass().getCanonicalName())); + } + + return Optional.of(tableName); + } + + public static String stringifyHints(List hints) { + StringBuilder sb = new StringBuilder(); + boolean first = true; + for (RelHint h : hints) { + if (h.hintName.equalsIgnoreCase(FlinkHints.HINT_ALIAS)) { + continue; + } + if (!first) { + sb.append(", "); + } + sb.append(h.hintName); + if (h.listOptions.size() > 0) { + String listStr = h.listOptions.stream().collect(Collectors.joining(",", "(", ")")); + sb.append(listStr); + } else if (h.kvOptions.size() > 0) { + String mapStr = + h.kvOptions.entrySet().stream() + .map(e -> e.getKey() + "=" + e.getValue()) + .collect(Collectors.joining(", ", "(", ")")); + sb.append(mapStr); + } + first = false; + } + return sb.toString(); + } + + /** Get all join hints. */ + public static List getAllJoinHints(List allHints) { + return allHints.stream() + .filter(hint -> JoinStrategy.isJoinStrategy(hint.hintName)) + .collect(Collectors.toList()); + } + + /** + * Get all query block alias hints. + * + *

Because query block alias hints will be propagated from root to leaves, so maybe one node + * will contain multi alias hints. But only the first one is the really query block name where + * this node is. + */ + public static List getQueryBlockAliasHints(List allHints) { + return allHints.stream() + .filter(hint -> hint.hintName.equals(FlinkHints.HINT_ALIAS)) + .collect(Collectors.toList()); + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/JoinStrategy.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/JoinStrategy.java new file mode 100644 index 0000000000000..cf6345167e04b --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/hint/JoinStrategy.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.hint; + +import java.util.List; + +/** Currently available join strategies and corresponding join hint names. */ +public enum JoinStrategy { + /** + * Instructs the optimizer to use broadcast hash join strategy. If both sides are specified in + * this hint, the side that is first written will be broadcast. + */ + BROADCAST("BROADCAST"), + + /** + * Instructs the optimizer to use shuffle hash join strategy. If both sides are specified in + * this hint, the side that is first written will be treated as the build side. + */ + SHUFFLE_HASH("SHUFFLE_HASH"), + + /** + * Instructs the optimizer to use shuffle sort merge join strategy. As long as one of the side + * is specified in this hint, it will be tried. + */ + SHUFFLE_MERGE("SHUFFLE_MERGE"), + + /** + * Instructs the optimizer to use nest loop join strategy. If both sides are specified in this + * hint, the side that is first written will be treated as the build side. + */ + NEST_LOOP("NEST_LOOP"); + + private final String joinHintName; + + JoinStrategy(String joinHintName) { + this.joinHintName = joinHintName; + } + + // ~ option name for join hint + public static final String LEFT_INPUT = "LEFT"; + public static final String RIGHT_INPUT = "RIGHT"; + + public static boolean isJoinStrategy(String hintName) { + try { + JoinStrategy.valueOf(hintName); + return true; + } catch (Exception e) { + return false; + } + } + + public String getJoinHintName() { + return joinHintName; + } + + public static boolean validOptions(String hintName, List options) { + if (!isJoinStrategy(hintName)) { + return false; + } + + JoinStrategy strategy = JoinStrategy.valueOf(hintName); + switch (strategy) { + case SHUFFLE_HASH: + case SHUFFLE_MERGE: + case BROADCAST: + case NEST_LOOP: + return options.size() > 0; + } + return false; + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolver.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolver.java new file mode 100644 index 0000000000000..177c3e3409067 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolver.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.optimize; + +import org.apache.flink.table.planner.hint.FlinkHints; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelShuttleImpl; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.hint.Hintable; +import org.apache.calcite.rel.hint.RelHint; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +/** A shuttle to remove query block alias hint. */ +public class ClearQueryBlockAliasResolver extends RelShuttleImpl { + + public List resolve(List roots) { + return roots.stream().map(node -> node.accept(this)).collect(Collectors.toList()); + } + + @Override + protected RelNode visitChild(RelNode parent, int i, RelNode child) { + RelNode newParent = clearQueryBlockAlias(parent); + return super.visitChild(newParent, i, child); + } + + @Override + public RelNode visit(TableScan scan) { + RelNode newScan = clearQueryBlockAlias(scan); + return super.visit(newScan); + } + + private RelNode clearQueryBlockAlias(RelNode relNode) { + if (!(relNode instanceof Hintable)) { + return relNode; + } + + List hints = ((Hintable) relNode).getHints(); + List newHints = new ArrayList<>(); + for (RelHint hint : hints) { + if (!FlinkHints.HINT_ALIAS.equals(hint.hintName)) { + newHints.add(hint); + } + } + + if (newHints.size() != hints.size()) { + return ((Hintable) relNode).withHints(newHints); + } + + return relNode; + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java new file mode 100644 index 0000000000000..407d2ce79ab7e --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolver.java @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.optimize; + +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.planner.hint.FlinkHints; +import org.apache.flink.table.planner.hint.JoinStrategy; + +import org.apache.calcite.rel.BiRel; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.RelShuttleImpl; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.hint.Hintable; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static java.util.Collections.singletonList; + +/** + * Resolve and validate the join hints. + * + *

Note: duplicate join hints are not checked here. + */ +public class JoinHintResolver extends RelShuttleImpl { + private final Set allHints = new HashSet<>(); + private final Set validHints = new HashSet<>(); + + /** + * Resolves and validates join hints in the given {@link RelNode} list, an {@link + * ValidationException} will be raised for invalid hints. + * + *

After resolving join hints, the options of the join hints (declared table name or query + * block name) will be replaced to {@link JoinStrategy#LEFT_INPUT} or {@link + * JoinStrategy#RIGHT_INPUT} + * + *

If the declared table name or query name in a join hint could not match the left side or + * right side of this join, that means this join hint is invalid and a {@link + * ValidationException} will be thrown. + */ + public List resolve(List roots) { + List 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 leftName = extractAliasOrTableName(biRel.getLeft()); + Optional rightName = extractAliasOrTableName(biRel.getRight()); + + Set existentKVHints = new HashSet<>(); + + List oldHints = ((Hintable) biRel).getHints(); + List newHints = new ArrayList<>(); + + for (RelHint hint : oldHints) { + if (JoinStrategy.isJoinStrategy(hint.hintName)) { + allHints.add(trimInheritPath(hint)); + // the declared table name or query block name is replaced by + // JoinStrategy#LEFT_INPUT or JoinStrategy#RIGHT_INPUT + List newOptions = + getNewJoinHintOptions(leftName, rightName, hint.listOptions, hint.hintName); + + // check whether the join hints options are valid + boolean isValidOption = JoinStrategy.validOptions(hint.hintName, newOptions); + if (isValidOption) { + validHints.add(trimInheritPath(hint)); + // if the hint defines more than one args, only + // retain the first one + newHints.add( + RelHint.builder(hint.hintName) + .hintOptions(singletonList(newOptions.get(0))) + .build()); + } + } else { + if (!existentKVHints.contains(hint)) { + existentKVHints.add(hint); + newHints.add(hint); + } + } + } + + RelNode newNode = super.visitChildren(biRel); + + List oldJoinHints = FlinkHints.getAllJoinHints(oldHints); + if (!oldJoinHints.isEmpty()) { + // replace the table name as LEFT or RIGHT + return ((Hintable) newNode).withHints(newHints); + } + // has no hints, return original node directly. + return newNode; + } + + private List getNewJoinHintOptions( + Optional leftName, + Optional rightName, + List listOptions, + String hintName) { + return listOptions.stream() + .map( + option -> { + if (leftName.isPresent() + && rightName.isPresent() + && matchIdentifier(option, leftName.get()) + && matchIdentifier(option, rightName.get())) { + throw new ValidationException( + String.format( + "Ambitious option: %s in hint: %s, the input " + + "relations are: %s, %s", + option, hintName, leftName, rightName)); + } else if (leftName.isPresent() + && matchIdentifier(option, leftName.get())) { + return JoinStrategy.LEFT_INPUT; + } else if (rightName.isPresent() + && matchIdentifier(option, rightName.get())) { + return JoinStrategy.RIGHT_INPUT; + } else { + return ""; + } + }) + .filter(StringUtils::isNotEmpty) + .collect(Collectors.toList()); + } + + private void validateHints() { + Set invalidHints = new HashSet<>(allHints); + invalidHints.removeAll(validHints); + if (!invalidHints.isEmpty()) { + String errorMsg = + invalidHints.stream() + .map( + hint -> + hint.hintName + + "(" + + StringUtils.join(hint.listOptions, ", ") + + ")`") + .collect(Collectors.joining("\n`", "\n`", "")); + throw new ValidationException( + String.format( + "The options of following hints cannot match the name of " + + "input tables or views: %s", + errorMsg)); + } + } + + private RelHint trimInheritPath(RelHint hint) { + RelHint.Builder builder = RelHint.builder(hint.hintName); + if (hint.listOptions.isEmpty()) { + return builder.hintOptions(hint.kvOptions).build(); + } else { + return builder.hintOptions(hint.listOptions).build(); + } + } + + private Optional extractAliasOrTableName(RelNode node) { + // check whether the input relation is converted from a view + Optional aliasName = FlinkHints.getTableAlias(node); + if (aliasName.isPresent()) { + return aliasName; + } + // otherwise, the option may be a table name + Optional tableScan = getTableScan(node); + if (tableScan.isPresent()) { + Optional tableName = FlinkHints.getTableName(tableScan.get().getTable()); + if (tableName.isPresent()) { + return tableName; + } + } + + return Optional.empty(); + } + + private Optional getTableScan(RelNode node) { + if (node instanceof TableScan) { + return Optional.of((TableScan) node); + } else { + if (FlinkHints.canTransposeToTableScan(node)) { + return getTableScan(node.getInput(0)); + } else { + return Optional.empty(); + } + } + } + + /** + * Check whether the given hint option matches the table qualified names. For convenience, we + * follow a simple rule: the matching is successful if the option is the suffix of the table + * qualified names. + */ + private boolean matchIdentifier(String option, String tableIdentifier) { + String[] optionNames = option.split("\\."); + int optionNameLength = optionNames.length; + + String[] tableNames = tableIdentifier.split("\\."); + int tableNameLength = tableNames.length; + + for (int i = 0; i < Math.min(optionNameLength, tableNameLength); i++) { + String currOptionName = optionNames[optionNameLength - 1 - i]; + String currTableName = tableNames[tableNameLength - 1 - i]; + + if (!currOptionName.equals(currTableName)) { + return false; + } + } + + return true; + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java index 225d270f382ad..2caa2bb951b92 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/StreamNonDeterministicUpdatePlanVisitor.java @@ -394,7 +394,8 @@ public StreamPhysicalRel visit( false, true, false, - true)); + true, + false)); throw new TableException(errorMsg.toString()); } } @@ -856,7 +857,8 @@ private void throwNonDeterministicConditionError( false, true, false, - true)); + true, + false)); throw new TableException(errorMsg.toString()); } @@ -897,7 +899,8 @@ private void throwNonDeterministicColumnsError( false, true, false, - true)); + true, + false)); throw new TableException(errorMsg.toString()); } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala index dde9120d942e1..41ca922b610b0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkLogicalRelFactories.scala @@ -165,7 +165,7 @@ object FlinkLogicalRelFactories { variablesSet: util.Set[CorrelationId], joinType: JoinRelType, semiJoinDone: Boolean): RelNode = { - FlinkLogicalJoin.create(left, right, condition, joinType) + FlinkLogicalJoin.create(left, right, condition, hints, joinType) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala index 7f8292284a7a3..f9488750b53eb 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkPlannerImpl.scala @@ -22,8 +22,10 @@ import org.apache.flink.sql.parser.ddl.{SqlCompilePlan, SqlReset, SqlSet, SqlUse import org.apache.flink.sql.parser.dml.{RichSqlInsert, SqlBeginStatementSet, SqlCompileAndExecutePlan, SqlEndStatementSet, SqlExecute, SqlExecutePlan, SqlStatementSet} import org.apache.flink.sql.parser.dql._ import org.apache.flink.table.api.{TableException, ValidationException} +import org.apache.flink.table.planner.hint.JoinStrategy import org.apache.flink.table.planner.parse.CalciteParser import org.apache.flink.table.planner.plan.FlinkCalciteCatalogReader +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil import com.google.common.collect.ImmutableList import org.apache.calcite.config.NullCollation @@ -33,8 +35,9 @@ import org.apache.calcite.rel.`type`.RelDataType import org.apache.calcite.rel.{RelFieldCollation, RelRoot} import org.apache.calcite.rel.hint.RelHint import org.apache.calcite.rex.{RexInputRef, RexNode} -import org.apache.calcite.sql.{SqlInsert, SqlKind, SqlNode, SqlOperatorTable} +import org.apache.calcite.sql.{SqlCall, SqlHint, SqlKind, SqlNode, SqlNodeList, SqlOperatorTable, SqlSelect, SqlTableRef} import org.apache.calcite.sql.advise.SqlAdvisorValidator +import org.apache.calcite.sql.util.SqlShuttle import org.apache.calcite.sql.validate.SqlValidator import org.apache.calcite.sql2rel.{SqlRexConvertletTable, SqlToRelConverter} import org.apache.calcite.tools.{FrameworkConfig, RelConversionException} @@ -192,6 +195,17 @@ class FlinkPlannerImpl( assert(validatedSqlNode != null) val sqlToRelConverter: SqlToRelConverter = createSqlToRelConverter(sqlValidator) + // check whether this SqlNode tree contains join hints + val checkContainJoinHintShuttle = new CheckContainJoinHintShuttle + validatedSqlNode.accept(checkContainJoinHintShuttle) + checkContainJoinHintShuttle.containsJoinHint + + // TODO currently, it is a relatively hacked way to tell converter + // that this SqlNode tree contains join hints + if (checkContainJoinHintShuttle.containsJoinHint) { + sqlToRelConverter.containsJoinHint() + } + sqlToRelConverter.convertQuery(validatedSqlNode, false, true) // we disable automatic flattening in order to let composite types pass without modification // we might enable it again once Calcite has better support for structured types @@ -206,6 +220,39 @@ class FlinkPlannerImpl( } } + class CheckContainJoinHintShuttle extends SqlShuttle { + var containsJoinHint: Boolean = false + + override def visit(call: SqlCall): SqlNode = { + call match { + case select: SqlSelect => + if (select.hasHints && hasJoinHint(select.getHints.getList)) { + containsJoinHint = true + return call + } + case table: SqlTableRef => + val hintList = table.getOperandList.get(1).asInstanceOf[SqlNodeList] + if (hasJoinHint(hintList.getList)) { + containsJoinHint = true + return call + } + case _ => // ignore + } + super.visit(call) + } + + private def hasJoinHint(hints: util.List[SqlNode]): Boolean = { + JavaScalaConversionUtil.toScala(hints).foreach { + case hint: SqlHint => + val hintName = hint.getName + if (JoinStrategy.isJoinStrategy(hintName)) { + return true + } + } + false + } + } + def validateExpression( sqlNode: SqlNode, inputRowType: RelDataType, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Sink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Sink.scala index c4d9d0bef1009..13a76581afc27 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Sink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/Sink.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.table.planner.plan.nodes.calcite -import org.apache.flink.table.catalog.{ContextResolvedTable, ObjectIdentifier, ResolvedCatalogTable} +import org.apache.flink.table.catalog.ContextResolvedTable import org.apache.flink.table.connector.sink.DynamicTableSink import org.apache.flink.table.planner.plan.utils.RelExplainUtil diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala index 0aff68e8980b9..8f1d42bd3c7bb 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalJoin.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.table.planner.plan.nodes.logical +import org.apache.flink.table.planner.JList import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.calcite.plan._ @@ -28,8 +29,6 @@ import org.apache.calcite.rel.logical.LogicalJoin import org.apache.calcite.rel.metadata.RelMetadataQuery import org.apache.calcite.rex.RexNode -import java.util.Collections - import scala.collection.JavaConversions._ /** @@ -42,16 +41,9 @@ class FlinkLogicalJoin( left: RelNode, right: RelNode, condition: RexNode, + hints: JList[RelHint], joinType: JoinRelType) - extends Join( - cluster, - traitSet, - Collections.emptyList[RelHint](), - left, - right, - condition, - Set.empty[CorrelationId], - joinType) + extends Join(cluster, traitSet, hints, left, right, condition, Set.empty[CorrelationId], joinType) with FlinkLogicalRel { override def copy( @@ -61,7 +53,7 @@ class FlinkLogicalJoin( right: RelNode, joinType: JoinRelType, semiJoinDone: Boolean): Join = { - new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, joinType) + new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, getHints, joinType) } override def computeSelfCost(planner: RelOptPlanner, mq: RelMetadataQuery): RelOptCost = { @@ -97,7 +89,7 @@ private class FlinkLogicalJoinConverter val join = rel.asInstanceOf[LogicalJoin] val newLeft = RelOptRule.convert(join.getLeft, FlinkConventions.LOGICAL) val newRight = RelOptRule.convert(join.getRight, FlinkConventions.LOGICAL) - FlinkLogicalJoin.create(newLeft, newRight, join.getCondition, join.getJoinType) + FlinkLogicalJoin.create(newLeft, newRight, join.getCondition, join.getHints, join.getJoinType) } } @@ -108,9 +100,10 @@ object FlinkLogicalJoin { left: RelNode, right: RelNode, conditionExpr: RexNode, + hints: JList[RelHint], joinType: JoinRelType): FlinkLogicalJoin = { val cluster = left.getCluster val traitSet = cluster.traitSetOf(FlinkConventions.LOGICAL).simplify() - new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, joinType) + new FlinkLogicalJoin(cluster, traitSet, left, right, conditionExpr, hints, joinType) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala index 107110b36fefe..a4d0baa1912fe 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.delegation.BatchPlanner import org.apache.flink.table.planner.plan.nodes.calcite.{LegacySink, Sink} import org.apache.flink.table.planner.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram} import org.apache.flink.table.planner.plan.schema.IntermediateRelTable +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil.{toJava, toScala} import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapContext import org.apache.flink.table.planner.utils.TableConfigUtils import org.apache.flink.util.Preconditions @@ -38,8 +39,20 @@ class BatchCommonSubGraphBasedOptimizer(planner: BatchPlanner) extends CommonSubGraphBasedOptimizer { override protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock] = { + // TODO currently join hint only works in BATCH + // resolve hints before optimizing + val joinHintResolver = new JoinHintResolver() + val resolvedHintRoots = joinHintResolver.resolve(toJava(roots)) + + // clear query block alias before optimizing + val clearQueryBlockAliasResolver = new ClearQueryBlockAliasResolver + val resolvedAliasRoots = clearQueryBlockAliasResolver.resolve(resolvedHintRoots) + // build RelNodeBlock plan - val rootBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, planner.getTableConfig) + val rootBlocks = + RelNodeBlockPlanBuilder.buildRelNodeBlockPlan( + toScala(resolvedAliasRoots), + planner.getTableConfig) // optimize recursively RelNodeBlock rootBlocks.foreach(optimizeBlock) rootBlocks diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala index 43f06975492b7..376717c84a6d9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitPythonConditionFromJoinRule.scala @@ -60,6 +60,7 @@ class SplitPythonConditionFromJoinRule join.getLeft, join.getRight, newJoinCondition, + join.getHints, join.getJoinType) val rexProgram = new RexProgramBuilder(bottomJoin.getRowType, rexBuilder).getProgram diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRule.scala index 21866df620da5..d3ef36ca15087 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/TemporalJoinRewriteWithUniqueKeyRule.scala @@ -110,7 +110,12 @@ class TemporalJoinRewriteWithUniqueKeyRule } }) val rewriteJoin = - FlinkLogicalJoin.create(leftInput, snapshot, newJoinCondition, join.getJoinType) + FlinkLogicalJoin.create( + leftInput, + snapshot, + newJoinCondition, + join.getHints, + join.getJoinType) call.transformTo(rewriteJoin) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala index 2eb6fb6ce3b65..4e22525e4491f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala @@ -17,16 +17,13 @@ */ package org.apache.flink.table.planner.plan.rules.physical.batch -import org.apache.flink.configuration.ReadableConfig -import org.apache.flink.table.api.config.OptimizerConfigOptions -import org.apache.flink.table.planner.JDouble +import org.apache.flink.table.api.TableException +import org.apache.flink.table.planner.hint.JoinStrategy import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistribution import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalHashJoin -import org.apache.flink.table.planner.plan.utils.OperatorType import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig -import org.apache.flink.table.planner.utils.TableConfigUtils.isOperatorDisabled import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} import org.apache.calcite.plan.RelOptRule.{any, operand} @@ -50,28 +47,15 @@ class BatchPhysicalHashJoinRule override def matches(call: RelOptRuleCall): Boolean = { val join: Join = call.rel(0) - val joinInfo = join.analyzeCondition - // join keys must not be empty - if (joinInfo.pairs().isEmpty) { - return false - } - - val tableConfig = unwrapTableConfig(call) - val isShuffleHashJoinEnabled = !isOperatorDisabled(tableConfig, OperatorType.ShuffleHashJoin) - val isBroadcastHashJoinEnabled = - !isOperatorDisabled(tableConfig, OperatorType.BroadcastHashJoin) + val tableConfig = unwrapTableConfig(join) - val leftSize = binaryRowRelNodeSize(join.getLeft) - val rightSize = binaryRowRelNodeSize(join.getRight) - val (isBroadcast, _) = canBroadcast(join.getJoinType, leftSize, rightSize, tableConfig) - - // TODO use shuffle hash join if isBroadcast is true and isBroadcastHashJoinEnabled is false ? - if (isBroadcast) isBroadcastHashJoinEnabled else isShuffleHashJoinEnabled + canUseJoinStrategy(join, tableConfig, JoinStrategy.BROADCAST) || + canUseJoinStrategy(join, tableConfig, JoinStrategy.SHUFFLE_HASH) } override def onMatch(call: RelOptRuleCall): Unit = { - val tableConfig = unwrapTableConfig(call) val join: Join = call.rel(0) + val tableConfig = unwrapTableConfig(join) val joinInfo = join.analyzeCondition val joinType = join.getJoinType @@ -89,19 +73,40 @@ class BatchPhysicalHashJoinRule case _ => (join.getRight, false) } - val leftSize = binaryRowRelNodeSize(left) - val rightSize = binaryRowRelNodeSize(right) - - val (isBroadcast, leftIsBroadcast) = canBroadcast(joinType, leftSize, rightSize, tableConfig) + val firstValidJoinHintOp = getFirstValidJoinHint(join, tableConfig) + + val (isBroadcast: Boolean, isLeftToBroadcastOrBuild: Boolean) = firstValidJoinHintOp match { + case Some(firstValidJoinHint) => + firstValidJoinHint match { + case JoinStrategy.BROADCAST => + val (_, isLeftToBroadcast: Boolean) = + checkBroadcast(join, tableConfig, withBroadcastHint = true) + (true, isLeftToBroadcast) + case JoinStrategy.SHUFFLE_HASH => + val (_, isLeftToBuild: Boolean) = + checkShuffleHash(join, tableConfig, withShuffleHashHint = true) + (false, isLeftToBuild) + case _ => + // this should not happen + throw new TableException( + String.format( + "The planner is trying to convert the " + + "`FlinkLogicalJoin` using BROADCAST or SHUFFLE_HASH," + + " but the first valid join hint is not BROADCAST or SHUFFLE_HASH: %s", + firstValidJoinHint + )) + } + case None => + // treat as non-join-hints + val (canBroadcast, isLeftToBroadcast) = + checkBroadcast(join, tableConfig, withBroadcastHint = false) - val leftIsBuild = if (isBroadcast) { - leftIsBroadcast - } else if (leftSize == null || rightSize == null || leftSize == rightSize) { - // use left to build hash table if leftSize or rightSize is unknown or equal size. - // choose right to build if join is SEMI/ANTI. - !join.getJoinType.projectsRight - } else { - leftSize < rightSize + if (canBroadcast) { + (true, isLeftToBroadcast) + } else { + val (_, isLeftToBuild) = checkShuffleHash(join, tableConfig, withShuffleHashHint = false) + (false, isLeftToBuild) + } } def transformToEquiv(leftRequiredTrait: RelTraitSet, rightRequiredTrait: RelTraitSet): Unit = { @@ -116,7 +121,7 @@ class BatchPhysicalHashJoinRule newRight, join.getCondition, join.getJoinType, - leftIsBuild, + isLeftToBroadcastOrBuild, isBroadcast, tryDistinctBuildRow) @@ -128,7 +133,7 @@ class BatchPhysicalHashJoinRule val buildTrait = join.getTraitSet .replace(FlinkConventions.BATCH_PHYSICAL) .replace(FlinkRelDistribution.BROADCAST_DISTRIBUTED) - if (leftIsBroadcast) { + if (isLeftToBroadcastOrBuild) { transformToEquiv(buildTrait, probeTrait) } else { transformToEquiv(probeTrait, buildTrait) @@ -157,40 +162,6 @@ class BatchPhysicalHashJoinRule } - /** - * Decides whether the join can convert to BroadcastHashJoin. - * - * @param joinType - * flink join type - * @param leftSize - * size of join left child - * @param rightSize - * size of join right child - * @return - * an Tuple2 instance. The first element of tuple is true if join can convert to broadcast hash - * join, false else. The second element of tuple is true if left side used as broadcast side, - * false else. - */ - private def canBroadcast( - joinType: JoinRelType, - leftSize: JDouble, - rightSize: JDouble, - tableConfig: ReadableConfig): (Boolean, Boolean) = { - // if leftSize or rightSize is unknown, cannot use broadcast - if (leftSize == null || rightSize == null) { - return (false, false) - } - val threshold = tableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD) - joinType match { - case JoinRelType.LEFT => (rightSize <= threshold, false) - case JoinRelType.RIGHT => (leftSize <= threshold, true) - case JoinRelType.FULL => (false, false) - case JoinRelType.INNER => - (leftSize <= threshold || rightSize <= threshold, leftSize < rightSize) - // left side cannot be used as build side in SEMI/ANTI join. - case JoinRelType.SEMI | JoinRelType.ANTI => (rightSize <= threshold, false) - } - } } object BatchPhysicalHashJoinRule { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala index a0af70041f143..94cf8884a67ad 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala @@ -20,19 +20,43 @@ package org.apache.flink.table.planner.plan.rules.physical.batch import org.apache.flink.annotation.Experimental import org.apache.flink.configuration.ConfigOption import org.apache.flink.configuration.ConfigOptions.key +import org.apache.flink.table.api.{TableConfig, TableException, ValidationException} +import org.apache.flink.table.api.config.OptimizerConfigOptions +import org.apache.flink.table.planner.JDouble +import org.apache.flink.table.planner.hint.JoinStrategy import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalLocalHashAggregate -import org.apache.flink.table.planner.plan.utils.FlinkRelMdUtil +import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, OperatorType} import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig +import org.apache.flink.table.planner.utils.TableConfigUtils.isOperatorDisabled import org.apache.calcite.plan.RelOptRule import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.core.{Join, JoinRelType} import org.apache.calcite.util.ImmutableBitSet import java.lang.{Boolean => JBoolean, Double => JDouble} +import scala.collection.JavaConversions._ + trait BatchPhysicalJoinRuleBase { + protected def canUseJoinStrategy( + join: Join, + tableConfig: TableConfig, + joinStrategy: JoinStrategy): Boolean = { + val firstValidJoinHint = getFirstValidJoinHint(join, tableConfig) + if (firstValidJoinHint.nonEmpty) { + // if there are join hints, the first hint must be this one, otherwise it is invalid + firstValidJoinHint.get.equals(joinStrategy) + } else { + // if there are no join hints, treat as non-join-hints + val (isValid, _) = + checkJoinStrategyValid(join, tableConfig, joinStrategy, withHint = false) + isValid + } + } + def addLocalDistinctAgg(node: RelNode, distinctKeys: Seq[Int]): RelNode = { val localRequiredTraitSet = node.getTraitSet.replace(FlinkConventions.BATCH_PHYSICAL) val newInput = RelOptRule.convert(node, localRequiredTraitSet) @@ -72,6 +96,238 @@ trait BatchPhysicalJoinRuleBase { rowCount * FlinkRelMdUtil.binaryRowAverageSize(relNode) } } + + protected def getFirstValidJoinHint( + join: Join, + tableConfig: TableConfig): Option[JoinStrategy] = { + val allHints = join.getHints + + allHints.forEach( + relHint => { + if (JoinStrategy.isJoinStrategy(relHint.hintName)) { + val joinStrategy = JoinStrategy.valueOf(relHint.hintName) + val (isValid, _) = + checkJoinStrategyValid(join, tableConfig, joinStrategy, withHint = true) + if (isValid) { + return Some(joinStrategy) + } + } + }) + + None + } + + /** + * Check whether the join strategy is valid. + * + * @param join + * the join node + * @param tableConfig + * the table config + * @param triedJoinStrategy + * the join strategy checked + * @param withHint + * whether this check is called with hint + * @return + * an Tuple2 instance. The first element of tuple is true if join is valid, false else. The + * second element of tuple is true if left side used as build side, false else. + */ + def checkJoinStrategyValid( + join: Join, + tableConfig: TableConfig, + triedJoinStrategy: JoinStrategy, + withHint: Boolean): (Boolean, Boolean) = { + + // TODO currently join hint is not supported with semi/anti join + if (withHint && !join.getJoinType.projectsRight()) { + return (false, false) + } + + triedJoinStrategy match { + case JoinStrategy.BROADCAST => + checkBroadcast(join, tableConfig, withHint) + + case JoinStrategy.SHUFFLE_HASH => + checkShuffleHash(join, tableConfig, withHint) + + case JoinStrategy.SHUFFLE_MERGE => + // for SortMergeJoin, there is no diff between with hint or without hint + // the second arg should be ignored + (checkSortMergeJoin(join, tableConfig), false) + + case JoinStrategy.NEST_LOOP => + checkNestLoopJoin(join, tableConfig, withHint) + + case _ => + throw new ValidationException("Unknown join strategy : " + triedJoinStrategy) + } + } + + private def isEquivJoin(join: Join): Boolean = { + val joinInfo = join.analyzeCondition + !joinInfo.pairs().isEmpty + } + + /** + * Decides whether the join can convert to BroadcastHashJoin. + * + * @param join + * the join node + * @return + * an Tuple2 instance. The first element of tuple is true if join can convert to broadcast hash + * join, false else. The second element of tuple is true if left side used as broadcast side, + * false else. + */ + protected def checkBroadcast( + join: Join, + tableConfig: TableConfig, + withBroadcastHint: Boolean): (Boolean, Boolean) = { + + if (!isEquivJoin(join) || isOperatorDisabled(tableConfig, OperatorType.BroadcastHashJoin)) { + return (false, false) + } + + // if it is with hint, try best to use it and only check the join type + if (withBroadcastHint) { + // BROADCAST use first arg as the broadcast side + val isLeftToBroadcastInHint = + getFirstArgInJoinHint(join, JoinStrategy.BROADCAST.getJoinHintName) + .equals(JoinStrategy.LEFT_INPUT) + + join.getJoinType match { + // if left join, must broadcast right side + case JoinRelType.LEFT => (!isLeftToBroadcastInHint, false) + // if right join, must broadcast left side + case JoinRelType.RIGHT => (isLeftToBroadcastInHint, true) + case JoinRelType.FULL => (false, false) + case JoinRelType.INNER => + (true, isLeftToBroadcastInHint) + case JoinRelType.SEMI | JoinRelType.ANTI => + // TODO currently join hint is not supported with semi/anti join + (false, false) + } + } else { + val leftSize = binaryRowRelNodeSize(join.getLeft) + val rightSize = binaryRowRelNodeSize(join.getRight) + + // if it is not with hint, just check size of left and right side by statistic and config + // if leftSize or rightSize is unknown, cannot use broadcast + if (leftSize == null || rightSize == null) { + return (false, false) + } + + val threshold = + tableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD) + + val rightSizeSmallerThanThreshold = rightSize <= threshold + val leftSizeSmallerThanThreshold = leftSize <= threshold + val leftSmallerThanRight = leftSize < rightSize + + join.getJoinType match { + case JoinRelType.LEFT => (rightSizeSmallerThanThreshold, false) + case JoinRelType.RIGHT => (leftSizeSmallerThanThreshold, true) + case JoinRelType.FULL => (false, false) + case JoinRelType.INNER => + ( + leftSizeSmallerThanThreshold + || rightSizeSmallerThanThreshold, + leftSmallerThanRight) + // left side cannot be used as build side in SEMI/ANTI join. + case JoinRelType.SEMI | JoinRelType.ANTI => + (rightSizeSmallerThanThreshold, false) + } + } + } + + protected def checkShuffleHash( + join: Join, + tableConfig: TableConfig, + withShuffleHashHint: Boolean): (Boolean, Boolean) = { + if (!isEquivJoin(join) || isOperatorDisabled(tableConfig, OperatorType.ShuffleHashJoin)) { + return (false, false) + } + + if (withShuffleHashHint) { + val isLeftToBuild = getFirstArgInJoinHint(join, JoinStrategy.SHUFFLE_HASH.getJoinHintName) + .equals(JoinStrategy.LEFT_INPUT) + (true, isLeftToBuild) + } else { + val leftSize = binaryRowRelNodeSize(join.getLeft) + val rightSize = binaryRowRelNodeSize(join.getRight) + val leftIsBuild = if (leftSize == null || rightSize == null || leftSize == rightSize) { + // use left to build hash table if leftSize or rightSize is unknown or equal size. + // choose right to build if join is SEMI/ANTI. + !join.getJoinType.projectsRight + } else { + leftSize < rightSize + } + (true, leftIsBuild) + + } + } + + // the sort merge join doesn't distinct the build side + protected def checkSortMergeJoin(join: Join, tableConfig: TableConfig): Boolean = { + if (!isEquivJoin(join) || isOperatorDisabled(tableConfig, OperatorType.SortMergeJoin)) { + false + } else { + true + } + } + + protected def checkNestLoopJoin( + join: Join, + tableConfig: TableConfig, + withNestLoopHint: Boolean): (Boolean, Boolean) = { + + if (isOperatorDisabled(tableConfig, OperatorType.NestedLoopJoin)) { + return (false, false) + } + + val isLeftToBuild = if (withNestLoopHint) { + getFirstArgInJoinHint(join, JoinStrategy.NEST_LOOP.getJoinHintName) + .equals(JoinStrategy.LEFT_INPUT) + } else { + join.getJoinType match { + case JoinRelType.LEFT => false + case JoinRelType.RIGHT => true + case JoinRelType.INNER | JoinRelType.FULL => + val leftSize = binaryRowRelNodeSize(join.getLeft) + val rightSize = binaryRowRelNodeSize(join.getRight) + // use left as build size if leftSize or rightSize is unknown. + if (leftSize == null || rightSize == null) { + true + } else { + leftSize <= rightSize + } + case JoinRelType.SEMI | JoinRelType.ANTI => false + } + + } + + // all join can use NEST LOOP JOIN + (true, isLeftToBuild) + + } + + private def getFirstArgInJoinHint(join: Join, joinHintName: String): String = { + join.getHints.forEach( + hint => { + if (hint.hintName.equals(joinHintName)) { + return hint.listOptions.get(0) + } + }) + + // can not happen + throw new TableException( + String.format( + "Fail to find the join hint `%s` among `%s`", + joinHintName, + join.getHints + .map(hint => hint.hintName) + .mkString(",") + )) + } } object BatchPhysicalJoinRuleBase { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalNestedLoopJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalNestedLoopJoinRule.scala index ba72542a32ff9..ca914cf69fd15 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalNestedLoopJoinRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalNestedLoopJoinRule.scala @@ -17,17 +17,19 @@ */ package org.apache.flink.table.planner.plan.rules.physical.batch +import org.apache.flink.table.api.TableException +import org.apache.flink.table.planner.hint.JoinStrategy import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalNestedLoopJoin -import org.apache.flink.table.planner.plan.utils.OperatorType import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig -import org.apache.flink.table.planner.utils.TableConfigUtils.isOperatorDisabled import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.plan.RelOptRule.{any, operand} import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.core.{Join, JoinRelType} +import scala.collection.JavaConversions._ + /** * Rule that converts [[FlinkLogicalJoin]] to [[BatchPhysicalNestedLoopJoin]] if NestedLoopJoin is * enabled. @@ -40,12 +42,14 @@ class BatchPhysicalNestedLoopJoinRule with BatchPhysicalNestedLoopJoinRuleBase { override def matches(call: RelOptRuleCall): Boolean = { - val tableConfig = unwrapTableConfig(call) - !isOperatorDisabled(tableConfig, OperatorType.NestedLoopJoin) + val join: Join = call.rel(0) + val tableConfig = unwrapTableConfig(join) + canUseJoinStrategy(join, tableConfig, JoinStrategy.NEST_LOOP) } override def onMatch(call: RelOptRuleCall): Unit = { val join: Join = call.rel(0) + val tableConfig = unwrapTableConfig(join) val left = join.getLeft val right = join.getJoinType match { case JoinRelType.SEMI | JoinRelType.ANTI => @@ -59,26 +63,33 @@ class BatchPhysicalNestedLoopJoinRule } case _ => join.getRight } - val leftIsBuild = isLeftBuild(join, left, right) - val newJoin = createNestedLoopJoin(join, left, right, leftIsBuild, singleRowJoin = false) - call.transformTo(newJoin) - } - private def isLeftBuild(join: Join, left: RelNode, right: RelNode): Boolean = { - join.getJoinType match { - case JoinRelType.LEFT => false - case JoinRelType.RIGHT => true - case JoinRelType.INNER | JoinRelType.FULL => - val leftSize = binaryRowRelNodeSize(left) - val rightSize = binaryRowRelNodeSize(right) - // use left as build size if leftSize or rightSize is unknown. - if (leftSize == null || rightSize == null) { - true - } else { - leftSize <= rightSize + val firstValidJoinHintOp = getFirstValidJoinHint(join, tableConfig) + + val temJoin = join.copy(join.getTraitSet, List(left, right)) + + val isLeftToBuild = firstValidJoinHintOp match { + case Some(firstValidJoinHint) => + firstValidJoinHint match { + case JoinStrategy.NEST_LOOP => + val (_, isLeft) = checkNestLoopJoin(temJoin, tableConfig, withNestLoopHint = true) + isLeft + case _ => + // this should not happen + throw new TableException(String.format( + "The planner is trying to convert the " + + "`FlinkLogicalJoin` using NEST_LOOP, but the valid join hint is not NEST_LOOP: %s", + firstValidJoinHint + )) } - case JoinRelType.SEMI | JoinRelType.ANTI => false + case None => + // treat as non-join-hints + val (_, isLeft) = checkNestLoopJoin(temJoin, tableConfig, withNestLoopHint = false) + isLeft } + + val newJoin = createNestedLoopJoin(join, left, right, isLeftToBuild, singleRowJoin = false) + call.transformTo(newJoin) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSingleRowJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSingleRowJoinRule.scala index 01625a8dda868..904802f20342c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSingleRowJoinRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSingleRowJoinRule.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.rules.physical.batch import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalNestedLoopJoin +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall} import org.apache.calcite.plan.volcano.RelSubset @@ -42,13 +43,21 @@ class BatchPhysicalSingleRowJoinRule override def matches(call: RelOptRuleCall): Boolean = { val join: Join = call.rel(0) - join.getJoinType match { - case JoinRelType.INNER | JoinRelType.FULL => - isSingleRow(join.getLeft) || isSingleRow(join.getRight) - case JoinRelType.LEFT => isSingleRow(join.getRight) - case JoinRelType.RIGHT => isSingleRow(join.getLeft) - case JoinRelType.SEMI | JoinRelType.ANTI => isSingleRow(join.getRight) - case _ => false + val tableConfig = unwrapTableConfig(join) + val firstValidJoinHintOp = getFirstValidJoinHint(join, tableConfig) + + firstValidJoinHintOp match { + // the valid join hint keeps higher priority + case Some(_) => false + case None => + join.getJoinType match { + case JoinRelType.INNER | JoinRelType.FULL => + isSingleRow(join.getLeft) || isSingleRow(join.getRight) + case JoinRelType.LEFT => isSingleRow(join.getRight) + case JoinRelType.RIGHT => isSingleRow(join.getLeft) + case JoinRelType.SEMI | JoinRelType.ANTI => isSingleRow(join.getRight) + case _ => false + } } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala index a43432a3ebb47..676a2aac6a020 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala @@ -20,13 +20,13 @@ package org.apache.flink.table.planner.plan.rules.physical.batch import org.apache.flink.annotation.Experimental import org.apache.flink.configuration.ConfigOption import org.apache.flink.configuration.ConfigOptions.key +import org.apache.flink.table.planner.hint.JoinStrategy import org.apache.flink.table.planner.plan.`trait`.FlinkRelDistribution import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalJoin import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalSortMergeJoin -import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, OperatorType} +import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig -import org.apache.flink.table.planner.utils.TableConfigUtils.isOperatorDisabled import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet} import org.apache.calcite.plan.RelOptRule.{any, operand} @@ -50,10 +50,8 @@ class BatchPhysicalSortMergeJoinRule override def matches(call: RelOptRuleCall): Boolean = { val join: Join = call.rel(0) - val joinInfo = join.analyzeCondition - val tableConfig = unwrapTableConfig(call) - val isSortMergeJoinEnabled = !isOperatorDisabled(tableConfig, OperatorType.SortMergeJoin) - !joinInfo.pairs().isEmpty && isSortMergeJoinEnabled + val tableConfig = unwrapTableConfig(join) + canUseJoinStrategy(join, tableConfig, JoinStrategy.SHUFFLE_MERGE) } override def onMatch(call: RelOptRuleCall): Unit = { @@ -107,7 +105,7 @@ class BatchPhysicalSortMergeJoinRule call.transformTo(newJoin) } - val tableConfig = unwrapTableConfig(call) + val tableConfig = unwrapTableConfig(join) val candidates = if (tableConfig.get(BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED)) { // add more possibility to remove redundant sort, and longer optimization time diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala index 848c990bf7b6e..a971a671fe3f2 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala @@ -70,7 +70,8 @@ object FlinkRelOptUtil { withIdPrefix: Boolean = false, withChangelogTraits: Boolean = false, withRowType: Boolean = false, - withUpsertKey: Boolean = false): String = { + withUpsertKey: Boolean = false, + withQueryBlockAlias: Boolean = false): String = { if (rel == null) { return null } @@ -82,7 +83,9 @@ object FlinkRelOptUtil { withChangelogTraits, withRowType, withTreeStyle = true, - withUpsertKey) + withUpsertKey, + withJoinHint = true, + withQueryBlockAlias) rel.explain(planWriter) sw.toString } @@ -143,7 +146,8 @@ object FlinkRelOptUtil { // expressions have different types withRowType = true, // ignore tree style, only contains RelNode's attributes - withTreeStyle = false)) + withTreeStyle = false, + withJoinHint = true)) sw.toString } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala index 1a64427e2ba69..9a7328b91c193 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RelTreeWriterImpl.scala @@ -17,11 +17,14 @@ */ package org.apache.flink.table.planner.plan.utils +import org.apache.flink.table.planner.hint.FlinkHints import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel import org.apache.calcite.rel.RelNode +import org.apache.calcite.rel.core.{Join, TableScan} import org.apache.calcite.rel.externalize.RelWriterImpl +import org.apache.calcite.rel.hint.Hintable import org.apache.calcite.sql.SqlExplainLevel import org.apache.calcite.util.Pair @@ -38,7 +41,9 @@ class RelTreeWriterImpl( withChangelogTraits: Boolean = false, withRowType: Boolean = false, withTreeStyle: Boolean = true, - withUpsertKey: Boolean = false) + withUpsertKey: Boolean = false, + withJoinHint: Boolean = true, + withQueryBlockAlias: Boolean = false) extends RelWriterImpl(pw, explainLevel, withIdPrefix) { var lastChildren: Seq[Boolean] = Nil @@ -103,6 +108,35 @@ class RelTreeWriterImpl( case _ => // ignore } + if (withJoinHint) { + rel match { + case join: Join => + val joinHints = FlinkHints.getAllJoinHints(join.getHints) + if (joinHints.nonEmpty) { + printValues.add(Pair.of("joinHints", RelExplainUtil.hintsToString(joinHints))) + } + case _ => // ignore + } + } + + if (withQueryBlockAlias) { + rel match { + case node: Hintable => + node match { + case _: TableScan => + // We don't need to pint hints about TableScan because TableScan will always + // print hints if exist. See more in such as LogicalTableScan#explainTerms + case _ => + val queryBlockAliasHints = FlinkHints.getQueryBlockAliasHints(node.getHints) + if (queryBlockAliasHints.nonEmpty) { + printValues.add( + Pair.of("hints", RelExplainUtil.hintsToString(queryBlockAliasHints))) + } + } + case _ => // ignore + } + } + if (!printValues.isEmpty) { var j = 0 printValues.toSeq.foreach { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java new file mode 100644 index 0000000000000..1239c46448b18 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.java @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.alias; + +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.GenericInMemoryCatalog; +import org.apache.flink.table.planner.calcite.FlinkRelBuilder; +import org.apache.flink.table.planner.hint.FlinkHints; +import org.apache.flink.table.planner.hint.JoinStrategy; +import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil; +import org.apache.flink.table.planner.utils.BatchTableTestUtil; +import org.apache.flink.table.planner.utils.PlannerMocks; +import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.table.utils.CatalogManagerMocks; + +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.sql.SqlExplainLevel; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; + +/** A test class for {@link ClearJoinHintWithInvalidPropagationShuttle}. */ +public class ClearJoinHintWithInvalidPropagationShuttleTest extends TableTestBase { + + private final BatchTableTestUtil util = batchTestUtil(TableConfig.getDefault()); + private final Catalog catalog = new GenericInMemoryCatalog("MockCatalog", "default"); + private final CatalogManager catalogManager = + CatalogManagerMocks.preparedCatalogManager() + .defaultCatalog("builtin", catalog) + .config( + Configuration.fromMap( + Collections.singletonMap( + ExecutionOptions.RUNTIME_MODE.key(), + RuntimeExecutionMode.BATCH.name()))) + .build(); + private final PlannerMocks plannerMocks = + PlannerMocks.newBuilder() + .withBatchMode(true) + .withCatalogManager(catalogManager) + .build(); + private final FlinkRelBuilder builder = plannerMocks.getPlannerContext().createRelBuilder(); + + @Before + public void before() throws Exception { + util.tableEnv().registerCatalog("testCatalog", catalog); + util.tableEnv().executeSql("use catalog testCatalog"); + + util.tableEnv() + .executeSql( + "CREATE TABLE t1 (\n" + + " a BIGINT\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + util.tableEnv() + .executeSql( + "CREATE TABLE t2 (\n" + + " a BIGINT\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + util.tableEnv() + .executeSql( + "CREATE TABLE t3 (\n" + + " a BIGINT\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + } + + @Test + public void testNoNeedToClearJoinHint() { + // SELECT /*+ BROADCAST(t1)*/t1.a FROM t1 JOIN t2 ON t1.a = t2.a + RelHint joinHintInView = + RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t1").build(); + + RelNode root = + builder.scan("t1") + .scan("t2") + .join( + JoinRelType.INNER, + builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a"))) + .project(builder.field(1, 0, "a")) + .hints(joinHintInView) + .build(); + verifyRelPlan(root); + } + + @Test + public void testClearJoinHintWithInvalidPropagationToViewWhileViewHasJoinHints() { + // SELECT /*+ BROADCAST(t3)*/t4.a FROM ( + // SELECT /*+ BROADCAST(t1)*/t1.a FROM t1 JOIN t2 ON t1.a = t2.a + // ) t4 JOIN t3 ON t4.a = t3.a + RelHint joinHintInView = + RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t1").build(); + + RelHint joinHintRoot = + RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t3").build(); + + RelHint aliasHint = RelHint.builder(FlinkHints.HINT_ALIAS).hintOption("t4").build(); + + RelNode root = + builder.scan("t1") + .scan("t2") + .join( + JoinRelType.INNER, + builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a"))) + .project(builder.field(1, 0, "a")) + .hints(joinHintInView, aliasHint) + .scan("t3") + .join( + JoinRelType.INNER, + builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a"))) + .project(builder.field(1, 0, "a")) + .hints(joinHintRoot) + .build(); + verifyRelPlan(root); + } + + @Test + public void testClearJoinHintWithInvalidPropagationToViewWhileViewHasNoJoinHints() { + // SELECT /*+ BROADCAST(t3)*/t4.a FROM ( + // SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a + // ) t4 JOIN t3 ON t4.a = t3.a + RelHint joinHintRoot = + RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t3").build(); + + RelHint aliasHint = RelHint.builder(FlinkHints.HINT_ALIAS).hintOption("t4").build(); + + RelNode root = + builder.scan("t1") + .scan("t2") + .join( + JoinRelType.INNER, + builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a"))) + .project(builder.field(1, 0, "a")) + .hints(aliasHint) + .scan("t3") + .join( + JoinRelType.INNER, + builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a"))) + .project(builder.field(1, 0, "a")) + .hints(joinHintRoot) + .build(); + verifyRelPlan(root); + } + + @Test + public void testClearJoinHintWithoutPropagatingToView() { + // SELECT /*+ BROADCAST(t1)*/t4.a FROM ( + // SELECT t1.a FROM t1 JOIN t2 ON t1.a = t2.a + // ) t4 JOIN t3 ON t4.a = t3.a + RelHint joinHintRoot = + RelHint.builder(JoinStrategy.BROADCAST.getJoinHintName()).hintOption("t1").build(); + + RelHint aliasHint = RelHint.builder(FlinkHints.HINT_ALIAS).hintOption("t4").build(); + + RelNode root = + builder.scan("t1") + .scan("t2") + .join( + JoinRelType.INNER, + builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a"))) + .project(builder.field(1, 0, "a")) + .hints(aliasHint) + .scan("t3") + .join( + JoinRelType.INNER, + builder.equals(builder.field(2, 0, "a"), builder.field(2, 1, "a"))) + .project(builder.field(1, 0, "a")) + .hints(joinHintRoot) + .build(); + verifyRelPlan(root); + } + + private String buildRelPlanWithQueryBlockAlias(RelNode node) { + return System.lineSeparator() + + FlinkRelOptUtil.toString( + node, SqlExplainLevel.EXPPLAN_ATTRIBUTES, false, false, true, false, true); + } + + private void verifyRelPlan(RelNode node) { + String plan = buildRelPlanWithQueryBlockAlias(node); + util.assertEqualsOrExpand("beforePropagatingHints", plan, true); + + RelNode rootAfterHintPropagation = RelOptUtil.propagateRelHints(node, false); + plan = buildRelPlanWithQueryBlockAlias(rootAfterHintPropagation); + util.assertEqualsOrExpand("afterPropagatingHints", plan, true); + + RelNode rootAfterClearingJoinHintWithInvalidPropagation = + rootAfterHintPropagation.accept(new ClearJoinHintWithInvalidPropagationShuttle()); + plan = buildRelPlanWithQueryBlockAlias(rootAfterClearingJoinHintWithInvalidPropagation); + util.assertEqualsOrExpand("afterClearingJoinHints", plan, false); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.java new file mode 100644 index 0000000000000..4a96fcd3bbc6b --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.batch.sql.join.joinhint; + +import org.apache.flink.table.planner.hint.JoinStrategy; + +/** Test for Broadcast join hint. */ +public class BroadcastJoinHintTest extends JoinHintTestBase { + + @Override + protected String getTestSingleJoinHint() { + return JoinStrategy.BROADCAST.getJoinHintName(); + } + + @Override + protected String getDisabledOperatorName() { + return "HashJoin"; + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/JoinHintTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/JoinHintTestBase.java new file mode 100644 index 0000000000000..d19f354751605 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/JoinHintTestBase.java @@ -0,0 +1,726 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.batch.sql.join.joinhint; + +import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.SqlParserException; +import org.apache.flink.table.api.StatementSet; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.planner.hint.JoinStrategy; +import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder; +import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil; +import org.apache.flink.table.planner.utils.BatchTableTestUtil; +import org.apache.flink.table.planner.utils.PlanKind; +import org.apache.flink.table.planner.utils.TableTestBase; + +import org.apache.flink.shaded.curator5.org.apache.curator.shaded.com.google.common.collect.Lists; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.sql.SqlExplainLevel; +import org.apache.logging.log4j.util.Strings; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.stream.Collectors; + +import scala.Enumeration; + +import static scala.runtime.BoxedUnit.UNIT; + +/** + * A test base for join hint. + * + *

TODO add test to cover legacy table source. + */ +public abstract class JoinHintTestBase extends TableTestBase { + + protected BatchTableTestUtil util; + + private final List allJoinHintNames = + Lists.newArrayList(JoinStrategy.values()).stream() + .map(JoinStrategy::getJoinHintName) + .collect(Collectors.toList()); + + @Before + public void before() { + util = batchTestUtil(TableConfig.getDefault()); + util.tableEnv() + .executeSql( + "CREATE TABLE T1 (\n" + + " a1 BIGINT,\n" + + " b1 VARCHAR\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + util.tableEnv() + .executeSql( + "CREATE TABLE T2 (\n" + + " a2 BIGINT,\n" + + " b2 VARCHAR\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + util.tableEnv() + .executeSql( + "CREATE TABLE T3 (\n" + + " a3 BIGINT,\n" + + " b3 VARCHAR\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + util.tableEnv().executeSql("CREATE View V4 as select a3 as a4, b3 as b4 from T3"); + } + + protected abstract String getTestSingleJoinHint(); + + protected abstract String getDisabledOperatorName(); + + protected void verifyRelPlanByCustom(String sql) { + util.doVerifyPlan( + sql, + new ExplainDetail[] {}, + false, + new Enumeration.Value[] {PlanKind.AST(), PlanKind.OPT_REL()}, + true); + } + + protected void verifyRelPlanByCustom(StatementSet set) { + util.doVerifyPlan( + set, + new ExplainDetail[] {}, + false, + new Enumeration.Value[] {PlanKind.AST(), PlanKind.OPT_REL()}, + () -> UNIT, + true); + } + + protected List getOtherJoinHints() { + return allJoinHintNames.stream() + .filter(name -> !name.equals(getTestSingleJoinHint())) + .collect(Collectors.toList()); + } + + @Test + public void testSimpleJoinHintWithLeftSideAsBuildSide() { + String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testSimpleJoinHintWithRightSideAsBuildSide() { + String sql = "select /*+ %s(T2) */* from T1 join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithMultiJoinAndFirstSideAsBuildSide1() { + // the T1 will be the build side in first join + String sql = + "select /*+ %s(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithMultiJoinAndFirstSideAsBuildSide2() { + String sql = + "select /*+ %s(T1, T2) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides1() { + String sql = + "select /*+ %s(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T1.b1 = T3.b3"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithMultiJoinAndSecondThirdSideAsBuildSides2() { + String sql = + "select /*+ %s(T2, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithMultiJoinAndFirstThirdSideAsBuildSides() { + String sql = + "select /*+ %s(T1, T3) */* from T1, T2, T3 where T1.a1 = T2.a2 and T2.b2 = T3.b3"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithUnknownTable() { + thrown().expect(ValidationException.class); + thrown().expectMessage( + "The options of following hints cannot match the name of input tables or views:"); + String sql = "select /*+ %s(T99) */* from T1 join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithView() { + String sql = "select /*+ %s(V4) */* from T1 join V4 on T1.a1 = V4.a4"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithUnknownView() { + thrown().expect(ValidationException.class); + thrown().expectMessage( + String.format( + "The options of following hints cannot match the name of input tables or views: \n" + + "`%s(V99)`", + getTestSingleJoinHint())); + String sql = "select /*+ %s(V99) */* from T1 join V4 on T1.a1 = V4.a4"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithEquiPred() { + String sql = "select /*+ %s(T1) */* from T1, T2 where T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithEquiPredAndFilter() { + String sql = "select /*+ %s(T1) */* from T1, T2 where T1.a1 = T2.a2 and T1.a1 > 1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithEquiAndLocalPred() { + String sql = "select /*+ %s(T1) */* from T1 inner join T2 on T1.a1 = T2.a2 and T1.a1 < 1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithEquiAndNonEquiPred() { + String sql = + "select /*+ %s(T1) */* from T1 inner join T2 on T1.b1 = T2.b2 and T1.a1 < 1 and T1.a1 < T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithoutJoinPred() { + String sql = "select /*+ %s(T1) */* from T1, T2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithNonEquiPred() { + String sql = "select /*+ %s(T1) */* from T1 inner join T2 on T1.a1 > T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithLeftJoinAndLeftSideAsBuildSide() { + String sql = "select /*+ %s(T1) */* from T1 left join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithLeftJoinAndRightSideAsBuildSide() { + String sql = "select /*+ %s(T2) */* from T1 left join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithRightJoinAndLeftSideAsBuildSide() { + String sql = "select /*+ %s(T1) */* from T1 right join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithRightJoinAndRightSideAsBuildSide() { + String sql = "select /*+ %s(T2) */* from T1 right join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithFullJoinAndLeftSideAsBuildSide() { + String sql = "select /*+ %s(T1) */* from T1 full join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithFullJoinAndRightSideAsBuildSide() { + String sql = "select /*+ %s(T2) */* from T1 full join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + // TODO currently join hint is not supported on SEMI join, it will use default join strategy by + // planner + @Test + public void testJoinHintWithSemiJoinAndLeftSideAsBuildSide() { + String sql = "select /*+ %s(T1) */* from T1 where a1 in (select a2 from T2)"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + // TODO currently join hint is not supported on SEMI join, it will use default join strategy by + // planner + @Test + public void testJoinHintWithSemiJoinAndRightSideAsBuildSide() { + String sql = "select /*+ %s(T2) */* from T1 where a1 in (select a2 from T2)"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + // TODO currently join hint is not supported on ANTI join, it will use default join strategy by + // planner + @Test + public void testJoinHintWithAntiJoinAndLeftSideAsBuildSide() { + String sql = "select /*+ %s(T1) */* from T1 where a1 not in (select a2 from T2)"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + // TODO currently join hint is not supported on ANTI join, it will use default join strategy by + // planner + @Test + public void testJoinHintWithAntiJoinAndRightSideAsBuildSide() { + String sql = "select /*+ %s(T2) */* from T1 where a1 not in (select a2 from T2)"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithMultiArgsAndLeftSideFirst() { + // the first arg will be chosen as the build side + String sql = "select /*+ %s(T1, T2) */* from T1 right join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithMultiArgsAndRightSideFirst() { + // the first arg will be chosen as the build side + String sql = "select /*+ %s(T2, T1) */* from T1 right join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testMultiJoinHints() { + // the first join hint will be chosen + String sql = "select /*+ %s(T1), %s */* from T1 join T2 on T1.a1 = T2.a2"; + + String otherJoinHints = + Strings.join( + getOtherJoinHints().stream() + .map(name -> String.format("%s(T1)", name)) + .collect(Collectors.toList()), + ','); + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), otherJoinHints)); + } + + @Test + public void testMultiJoinHintsWithTheFirstOneIsInvalid() { + // the first join hint is invalid because it is not equi join except NEST_LOOP + String sql = "select /*+ %s(T1), NEST_LOOP(T1) */* from T1 join T2 on T1.a1 > T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithoutAffectingJoinInView() { + // the join in V2 will use the planner's default join strategy, + // and the join between T1 and V2 will use BROADCAST + util.tableEnv() + .executeSql("create view V2 as select T1.* from T1 join T2 on T1.a1 = T2.a2"); + + String sql = "select /*+ %s(T1)*/T1.* from T1 join V2 on T1.a1 = V2.a1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithoutAffectingJoinInSubQuery() { + // the join in sub-query will use the planner's default join strategy, + // and the join outside will use BROADCAST + String sql = + "select /*+ %s(T1)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithTableAlias() { + // the join in sub-query will use the planner's default join strategy, + // and the join between T1 and alias V2 will use BROADCAST + String sql = + "select /*+ %s(V2)*/T1.* from T1 join (select T1.* from T1 join T2 on T1.a1 = T2.a2) V2 on T1.a1 = V2.a1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintsWithMultiSameJoinHintsAndSingleArg() { + // the first join hint will be chosen and T1 will be chosen as the build side + String sql = "select /*+ %s(T1), %s(T2) */* from T1 join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint())); + } + + @Test + public void testJoinHintsWithDuplicatedArgs() { + // T1 will be chosen as the build side + String sql = "select /*+ %s(T1, T1) */* from T1 join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint())); + } + + @Test + public void testJoinHintsWithMultiSameJoinHintsAndMultiArgs() { + // the first join hint will be chosen and T1 will be chosen as the build side + String sql = "select /*+ %s(T1, T2), %s(T2, T1) */* from T1 join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint())); + } + + @Test + public void testJoinHintsWithMultiHintsThrowException() { + thrown().expect(SqlParserException.class); + thrown().expectMessage("SQL parse failed."); + String sql = "select /*+ %s(T1) */ /*+ %s(T2) */ * from T1 join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithDisabledOperator() { + util.tableEnv() + .getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, + getDisabledOperatorName()); + + String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintsWithUnion() { + // there are two query blocks and join hints are independent + String sql = + "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2 union select /*+ %s(T3) */* from T3 join T1 on T3.a3 = T1.a1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint(), getTestSingleJoinHint())); + } + + @Test + public void testJoinHintsWithFilter() { + // there are two query blocks and join hints are independent + String sql = "select /*+ %s(T1) */* from T1 join T2 on T1.a1 = T2.a2 where T1.a1 > 5"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintsWithCalc() { + // there are two query blocks and join hints are independent + String sql = "select /*+ %s(T1) */a1 + 1, a1 * 10 from T1 join T2 on T1.a1 = T2.a2"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintInView() { + // the build side in view is left + util.tableEnv() + .executeSql( + String.format( + "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2", + getTestSingleJoinHint())); + + // the build side outside is right + String sql = "select /*+ %s(V2)*/T3.* from T3 join V2 on T3.a3 = V2.a1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintInMultiLevelView() { + // the inside view keeps multi alias + // the build side in this view is left + util.tableEnv() + .executeSql( + String.format( + "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2", + getTestSingleJoinHint())); + + // the build side in this view is right + util.tableEnv() + .executeSql( + String.format( + "create view V3 as select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1", + getTestSingleJoinHint())); + + // the build side outside is left + String sql = "select /*+ %s(V3)*/V3.* from V3 join T1 on V3.a1 = T1.a1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintsOnSameViewWithoutReusingView() { + // the build side in this view is left + util.tableEnv() + .executeSql( + String.format( + "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2", + getTestSingleJoinHint())); + + util.tableEnv() + .executeSql( + "CREATE TABLE S1 (\n" + + " a1 BIGINT,\n" + + " b1 VARCHAR\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + util.tableEnv() + .executeSql( + "CREATE TABLE S2 (\n" + + " a1 BIGINT,\n" + + " b1 VARCHAR\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + StatementSet set = util.tableEnv().createStatementSet(); + + // the calc will be pushed down + set.addInsertSql( + String.format( + "insert into S1 select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 2", + getTestSingleJoinHint())); + set.addInsertSql( + String.format( + "insert into S2 select /*+ %s(T1)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 5", + getTestSingleJoinHint())); + + verifyRelPlanByCustom(set); + } + + @Test + public void testJoinHintsOnSameViewWithReusingView() { + util.tableEnv() + .getConfig() + .set( + RelNodeBlockPlanBuilder + .TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED(), + true); + + // the build side in this view is left + util.tableEnv() + .executeSql( + String.format( + "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2", + getTestSingleJoinHint())); + + util.tableEnv() + .executeSql( + "CREATE TABLE S1 (\n" + + " a1 BIGINT,\n" + + " b1 VARCHAR\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + util.tableEnv() + .executeSql( + "CREATE TABLE S2 (\n" + + " a1 BIGINT,\n" + + " b1 VARCHAR\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + StatementSet set = util.tableEnv().createStatementSet(); + + // the calc will be pushed down because the view has same digest + set.addInsertSql( + String.format( + "insert into S1 select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 2", + getTestSingleJoinHint())); + set.addInsertSql( + String.format( + "insert into S2 select /*+ %s(T1)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 5", + getTestSingleJoinHint())); + + verifyRelPlanByCustom(set); + } + + @Test + public void testJoinHintsOnSameViewWithoutReusingViewBecauseDifferentJoinHints() { + util.tableEnv() + .getConfig() + .set( + RelNodeBlockPlanBuilder + .TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED(), + true); + + // the build side in this view is left + util.tableEnv() + .executeSql( + String.format( + "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2", + getTestSingleJoinHint())); + + // the build side in this view is left + // V2 and V3 have different join hints + util.tableEnv() + .executeSql( + String.format( + "create view V3 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2", + getOtherJoinHints().get(0))); + + util.tableEnv() + .executeSql( + "CREATE TABLE S1 (\n" + + " a1 BIGINT,\n" + + " b1 VARCHAR\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + util.tableEnv() + .executeSql( + "CREATE TABLE S2 (\n" + + " a1 BIGINT,\n" + + " b1 VARCHAR\n" + + ") WITH (\n" + + " 'connector' = 'values',\n" + + " 'bounded' = 'true'\n" + + ")"); + + StatementSet set = util.tableEnv().createStatementSet(); + + // the calc will not be pushed down because the view has different digest + set.addInsertSql( + String.format( + "insert into S1 select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1 where V2.a1 > 2", + getTestSingleJoinHint())); + set.addInsertSql( + String.format( + "insert into S2 select /*+ %s(T1)*/ T1.* from T1 join V3 on T1.a1 = V3.a1 where V3.a1 > 5", + getOtherJoinHints().get(0))); + + verifyRelPlanByCustom(set); + } + + @Test + public void testJoinHintWithSubStringViewName1() { + util.tableEnv() + .executeSql( + String.format( + "create view V2 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2", + getTestSingleJoinHint())); + + // the build side in this view is right + util.tableEnv() + .executeSql( + String.format( + "create view V22 as select /*+ %s(V2)*/ T1.* from T1 join V2 on T1.a1 = V2.a1", + getTestSingleJoinHint())); + + // the build side outside is left + String sql = "select /*+ %s(V22)*/V22.* from V22 join T1 on V22.a1 = T1.a1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + @Test + public void testJoinHintWithSubStringViewName2() { + util.tableEnv() + .executeSql( + String.format( + "create view V22 as select /*+ %s(T1)*/ T1.* from T1 join T2 on T1.a1 = T2.a2", + getTestSingleJoinHint())); + + // the build side in this view is right + util.tableEnv() + .executeSql( + String.format( + "create view V2 as select /*+ %s(V22)*/ T1.* from T1 join V22 on T1.a1 = V22.a1", + getTestSingleJoinHint())); + + // the build side outside is left + String sql = "select /*+ %s(V2)*/V2.* from V2 join T1 on V2.a1 = T1.a1"; + + verifyRelPlanByCustom(String.format(sql, getTestSingleJoinHint())); + } + + protected String buildAstPlanWithQueryBlockAlias(List relNodes) { + StringBuilder astBuilder = new StringBuilder(); + relNodes.forEach( + node -> + astBuilder + .append(System.lineSeparator()) + .append( + FlinkRelOptUtil.toString( + node, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, + false, + false, + true, + false, + true))); + return astBuilder.toString(); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.java new file mode 100644 index 0000000000000..5c8b1e8f3e8dd --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.batch.sql.join.joinhint; + +import org.apache.flink.table.planner.hint.JoinStrategy; + +/** Test for Nest Loop join hint. */ +public class NestLoopJoinHintTest extends JoinHintTestBase { + + @Override + protected String getTestSingleJoinHint() { + return JoinStrategy.NEST_LOOP.getJoinHintName(); + } + + @Override + protected String getDisabledOperatorName() { + return "NestedLoopJoin"; + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.java new file mode 100644 index 0000000000000..bd19dcc53eb21 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.batch.sql.join.joinhint; + +import org.apache.flink.table.planner.hint.JoinStrategy; + +/** Test for Shuffle Hash join hint. */ +public class ShuffleHashJoinHintTest extends JoinHintTestBase { + + @Override + protected String getTestSingleJoinHint() { + return JoinStrategy.SHUFFLE_HASH.getJoinHintName(); + } + + @Override + protected String getDisabledOperatorName() { + return "HashJoin"; + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.java new file mode 100644 index 0000000000000..0cb3181218425 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.batch.sql.join.joinhint; + +import org.apache.flink.table.planner.hint.JoinStrategy; + +/** Test for Shuffle Merge join hint. */ +public class ShuffleMergeJoinHintTest extends JoinHintTestBase { + + @Override + protected String getTestSingleJoinHint() { + return JoinStrategy.SHUFFLE_MERGE.getJoinHintName(); + } + + @Override + protected String getDisabledOperatorName() { + return "SortMergeJoin"; + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java new file mode 100644 index 0000000000000..6b6d1a3fad681 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.optimize; + +import org.apache.flink.table.api.StatementSet; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.internal.StatementSetImpl; +import org.apache.flink.table.planner.hint.JoinStrategy; +import org.apache.flink.table.planner.plan.batch.sql.join.joinhint.JoinHintTestBase; +import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.table.planner.utils.TableTestUtil; + +import org.apache.calcite.rel.RelNode; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +/** A test class for {@link ClearQueryBlockAliasResolver}. */ +public class ClearQueryBlockAliasResolverTest extends JoinHintTestBase { + + // use any join hint for test + @Override + protected String getTestSingleJoinHint() { + return JoinStrategy.BROADCAST.getJoinHintName(); + } + + @Override + protected String getDisabledOperatorName() { + return "HashJoin"; + } + + /** + * Customize logic to verify the RelNode tree by sql. + * + *

Currently, mainly copy from {@link TableTestBase} and customize it. + */ + @Override + protected void verifyRelPlanByCustom(String sql) { + Table table = util.tableEnv().sqlQuery(sql); + RelNode relNode = TableTestUtil.toRelNode(table); + verifyRelPlanAfterResolverWithSql(sql, Collections.singletonList(relNode)); + } + + /** + * Customize logic to verify the RelNode tree by StatementSet. + * + *

Currently, mainly copy from {@link TableTestBase} and customize it. + */ + @Override + protected void verifyRelPlanByCustom(StatementSet set) { + StatementSetImpl testStmtSet = (StatementSetImpl) set; + + List relNodes = + testStmtSet.getOperations().stream() + .map(node -> util.getPlanner().translateToRel(node)) + .collect(Collectors.toList()); + verifyRelPlanAfterResolverWithStatementSet(relNodes); + } + + /** + * Customize logic to verify the RelNode tree. + * + *

Currently, mainly copy from {@link TableTestBase} and customize it. + */ + private void verifyRelPlanAfterResolverWithSql(String sql, List relNodes) { + relNodes = clearQueryBlockAlias(relNodes); + String astPlan = buildAstPlanWithQueryBlockAlias(relNodes); + + util.assertEqualsOrExpand("sql", sql, true); + util.assertEqualsOrExpand("ast", astPlan, false); + } + + private void verifyRelPlanAfterResolverWithStatementSet(List relNodes) { + relNodes = clearQueryBlockAlias(relNodes); + String astPlan = buildAstPlanWithQueryBlockAlias(relNodes); + + util.assertEqualsOrExpand("ast", astPlan, false); + } + + private List clearQueryBlockAlias(List relNodes) { + JoinHintResolver joinHintResolver = new JoinHintResolver(); + relNodes = joinHintResolver.resolve(relNodes); + ClearQueryBlockAliasResolver clearQueryBlockAliasResolver = + new ClearQueryBlockAliasResolver(); + return clearQueryBlockAliasResolver.resolve(relNodes); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java new file mode 100644 index 0000000000000..bc458317c023e --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.optimize; + +import org.apache.flink.table.api.StatementSet; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.internal.StatementSetImpl; +import org.apache.flink.table.planner.hint.JoinStrategy; +import org.apache.flink.table.planner.plan.batch.sql.join.joinhint.JoinHintTestBase; +import org.apache.flink.table.planner.utils.TableTestBase; +import org.apache.flink.table.planner.utils.TableTestUtil; + +import org.apache.calcite.rel.RelNode; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +/** A test class for {@link JoinHintResolver}. */ +public class JoinHintResolverTest extends JoinHintTestBase { + + // use any join hint for test + @Override + protected String getTestSingleJoinHint() { + return JoinStrategy.BROADCAST.getJoinHintName(); + } + + @Override + protected String getDisabledOperatorName() { + return "HashJoin"; + } + + /** + * Customize logic to verify the RelNode tree by sql. + * + *

Currently, mainly copy from {@link TableTestBase} and customize it. + */ + @Override + protected void verifyRelPlanByCustom(String sql) { + Table table = util.tableEnv().sqlQuery(sql); + RelNode relNode = TableTestUtil.toRelNode(table); + verifyRelPlanAfterResolverWithSql(sql, Collections.singletonList(relNode)); + } + + /** + * Customize logic to verify the RelNode tree by StatementSet. + * + *

Currently, mainly copy from {@link TableTestBase} and customize it. + */ + @Override + protected void verifyRelPlanByCustom(StatementSet set) { + StatementSetImpl testStmtSet = (StatementSetImpl) set; + + List relNodes = + testStmtSet.getOperations().stream() + .map(node -> util.getPlanner().translateToRel(node)) + .collect(Collectors.toList()); + verifyRelPlanAfterResolverWithStatementSet(relNodes); + } + + /** + * Customize logic to verify the RelNode tree. + * + *

Currently, mainly copy from {@link TableTestBase} and customize it. + */ + private void verifyRelPlanAfterResolverWithSql(String sql, List relNodes) { + relNodes = resolveJoinHint(relNodes); + String astPlan = buildAstPlanWithQueryBlockAlias(relNodes); + + util.assertEqualsOrExpand("sql", sql, true); + util.assertEqualsOrExpand("ast", astPlan, false); + } + + private void verifyRelPlanAfterResolverWithStatementSet(List relNodes) { + relNodes = resolveJoinHint(relNodes); + String astPlan = buildAstPlanWithQueryBlockAlias(relNodes); + + util.assertEqualsOrExpand("ast", astPlan, false); + } + + private List resolveJoinHint(List relNodes) { + JoinHintResolver joinHintResolver = new JoinHintResolver(); + return joinHintResolver.resolve(relNodes); + } +} diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.xml new file mode 100644 index 0000000000000..a5d50095e04c3 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/alias/ClearJoinHintWithInvalidPropagationShuttleTest.xml @@ -0,0 +1,150 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml index f8fc67e64575b..8fcc68cdde5d3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.xml @@ -432,6 +432,38 @@ Calc(select=[c, g]) +- Exchange(distribution=[hash[a]]) +- Calc(select=[a, c]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) +]]> + + + + + 10) + ]]> + + + ($2, 10))], joinType=[left]) + :- LogicalProject(k=[$0], v=[$1]) + : +- LogicalFilter(condition=[=($0, 0)]) + : +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) + +- LogicalProject(k=[$0], v=[$1]) + +- LogicalFilter(condition=[=($0, 0)]) + +- LogicalTableScan(table=[[default_catalog, default_database, src, source: [TestTableSource(k, v)]]]) +]]> + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.xml new file mode 100644 index 0000000000000..dc60bb7e78463 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/BroadcastJoinHintTest.xml @@ -0,0 +1,1210 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- Calc(select=[a1], where=[>(a1, 5)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) +]]> + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a2]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left]) + :- Exchange(distribution=[hash[a1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left]) + :- Exchange(distribution=[hash[a1]]) + : +- Calc(select=[a1], where=[>(a1, 5)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Exchange(distribution=[hash[a2]]) + +- Calc(select=[a2]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + +- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1], where=[>(a1, 5)]) + +- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + 5]]> + + + ($0, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 5)]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1]]> + + + ($0, 1))]) + +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 1)]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, a2)], select=[a1, b1, a2, b2], build=[left]) +:- Exchange(distribution=[broadcast]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, a2)], select=[a1, b1, a2, b2], build=[left]) +:- Exchange(distribution=[broadcast]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.xml new file mode 100644 index 0000000000000..3dc576a00298c --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/NestLoopJoinHintTest.xml @@ -0,0 +1,1206 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1]) + +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- Calc(select=[a1], where=[>(a1, 5)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) +]]> + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a2]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- Calc(select=[a1], where=[>(a1, 5)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a2]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + +- Calc(select=[a1, b1]) + +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1], where=[>(a1, 5)]) + +- Calc(select=[a1, b1]) + +- NestedLoopJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + 5]]> + + + ($0, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 5)]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1]]> + + + ($0, 1))]) + +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 1)]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, a2)], select=[a1, b1, a2, b2], build=[left]) +:- Exchange(distribution=[broadcast]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[NEST_LOOP inheritPath:[0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, a2)], select=[a1, b1, a2, b2], build=[left]) +:- Exchange(distribution=[broadcast]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.xml new file mode 100644 index 0000000000000..67f9ad612d40e --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleHashJoinHintTest.xml @@ -0,0 +1,1246 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1]) + +- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left]) + :- Exchange(distribution=[hash[a1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], build=[left]) + :- Exchange(distribution=[hash[a1]]) + : +- Calc(select=[a1], where=[>(a1, 5)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1]) + +- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) +]]> + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Exchange(distribution=[hash[a2]]) + +- Calc(select=[a2]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- Calc(select=[a1], where=[>(a1, 5)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a2]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + +- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], build=[left]) + :- Exchange(distribution=[hash[a1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], build=[left]) + :- Exchange(distribution=[hash[a1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Exchange(distribution=[hash[a1]]) + +- Calc(select=[a1], where=[>(a1, 5)]) + +- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2], build=[left]) + :- Exchange(distribution=[hash[a1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + 5]]> + + + ($0, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 5)]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1]) ++- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1]]> + + + ($0, 1))]) + +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 1)]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1]) ++- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, a2)], select=[a1, b1, a2, b2], build=[left]) +:- Exchange(distribution=[broadcast]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, a2)], select=[a1, b1, a2, b2], build=[left]) +:- Exchange(distribution=[broadcast]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.xml new file mode 100644 index 0000000000000..874d00d39ea25 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/join/joinhint/ShuffleMergeJoinHintTest.xml @@ -0,0 +1,1246 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1]) + +- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10]) + :- Exchange(distribution=[hash[a1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1]) + +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2]) + :- Exchange(distribution=[hash[a1]]) + : +- Calc(select=[a1], where=[>(a1, 5)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[], project=[a1], metadata=[]]], fields=[a1]) + +- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) +]]> + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Exchange(distribution=[hash[a2]]) + +- Calc(select=[a2]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a1]) + +- HashJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, a2], isBroadcast=[true], build=[left]) + :- Exchange(distribution=[broadcast]) + : +- Calc(select=[a1], where=[>(a1, 5)]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Calc(select=[a2]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + ($2, 2)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- LogicalProject(a1=[$0], b1=[$1]) + +- LogicalFilter(condition=[>($2, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 2)]) + +- Calc(select=[a1, b1]) + +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2]) + :- Exchange(distribution=[hash[a1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) + +Sink(table=[default_catalog.default_database.S2], fields=[a1, b1]) ++- Calc(select=[a1, b1]) + +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a10)], select=[a1, b1, a10]) + :- Exchange(distribution=[hash[a1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Exchange(distribution=[hash[a1]]) + +- Calc(select=[a1], where=[>(a1, 5)]) + +- Calc(select=[a1, b1]) + +- SortMergeJoin(joinType=[InnerJoin], where=[=(a1, a2)], select=[a1, b1, a2]) + :- Exchange(distribution=[hash[a1]]) + : +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) + +- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2, project=[a2], metadata=[]]], fields=[a2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + 5]]> + + + ($0, 5)]) + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 5)]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1]) ++- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1]]> + + + ($0, 1))]) + +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0, 0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, 1)]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1, filter=[]]], fields=[a1, b1]) ++- Exchange(distribution=[hash[a2]]) + +- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, a2)], select=[a1, b1, a2, b2], build=[left]) +:- Exchange(distribution=[broadcast]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_MERGE inheritPath:[0] options:[T1]][NEST_LOOP inheritPath:[0] options:[T1]]]]) + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]) + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]) +]]> + + + (a1, a2)], select=[a1, b1, a2, b2], build=[left]) +:- Exchange(distribution=[broadcast]) +: +- TableSourceScan(table=[[default_catalog, default_database, T1]], fields=[a1, b1]) ++- TableSourceScan(table=[[default_catalog, default_database, T2]], fields=[a2, b2]) +]]> + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml new file mode 100644 index 0000000000000..6408c6f0b5495 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/ClearQueryBlockAliasResolverTest.xml @@ -0,0 +1,702 @@ + + + + + + + + + + + + + + + + + + + + + + ($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] ++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + ($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] ++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + ($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] ++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + + + 5]]> + + + ($0, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1]]> + + + ($0, 1))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][NEST_LOOP options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.xml new file mode 100644 index 0000000000000..e75c03efacca9 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/optimize/JoinHintResolverTest.xml @@ -0,0 +1,702 @@ + + + + + + + + + + + + + + + + + + + + + + ($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] ++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + ($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] ++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V3]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[SHUFFLE_HASH options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V3]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + ($2, 2)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[RIGHT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] + +LogicalSink(table=[default_catalog.default_database.S2], fields=[a1, b1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] ++- LogicalProject(a1=[$0], b1=[$1]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalFilter(condition=[>($2, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a10, VARCHAR(2147483647) b10)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalProject(a1=[$0], b1=[$1], hints=[[[ALIAS options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]], hints=[[[ALIAS inheritPath:[0] options:[V2]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + + + 5]]> + + + ($0, 5)]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + +- LogicalJoin(condition=[=($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 1]]> + + + ($0, 1))]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + +- LogicalJoin(condition=[true], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T2.a2]]> + + + ($0, $2)], joinType=[inner], joinHints=[[[BROADCAST options:[LEFT]][NEST_LOOP options:[LEFT]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1, BIGINT a2, VARCHAR(2147483647) b2)] + :- LogicalTableScan(table=[[default_catalog, default_database, T1]], hints=[[[ALIAS inheritPath:[] options:[T1]]]]), rowType=[RecordType(BIGINT a1, VARCHAR(2147483647) b1)] + +- LogicalTableScan(table=[[default_catalog, default_database, T2]], hints=[[[ALIAS inheritPath:[] options:[T2]]]]), rowType=[RecordType(BIGINT a2, VARCHAR(2147483647) b2)] +]]> + + + + + + + + + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala index 42099c58e1496..dc2500d024255 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala @@ -107,12 +107,4 @@ class ShuffledHashJoinTest extends JoinTestBase { thrown.expectMessage("Cannot generate a valid execution plan for the given query") super.testCrossJoin() } - - @Test - override def testSelfJoin(): Unit = { - // TODO use shuffle hash join if isBroadcast is true and isBroadcastHashJoinEnabled is false ? - thrown.expect(classOf[TableException]) - thrown.expectMessage("Cannot generate a valid execution plan for the given query") - super.testSelfJoin() - } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryTestBase.scala index cfe212e5b2325..9204b27a4a617 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/subquery/SubQueryTestBase.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.rules.logical.subquery import org.apache.flink.table.planner.calcite.CalciteConfig +import org.apache.flink.table.planner.hint.FlinkHintStrategies import org.apache.flink.table.planner.plan.optimize.program.FlinkBatchProgram import org.apache.flink.table.planner.utils.{BatchTableTestUtil, TableConfigUtils, TableTestBase} @@ -35,7 +36,8 @@ class SubQueryTestBase extends TableTestBase { .config() .withTrimUnusedFields(false) .withExpand(false) - .withInSubQueryThreshold(3)) + .withInSubQueryThreshold(3) + .withHintStrategyTable(FlinkHintStrategies.createHintStrategyTable())) util.tableEnv.getConfig.setPlannerConfig(builder.build()) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index 1b37f8dcb86b9..fd2fd085519bc 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -300,7 +300,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) query, Array.empty[ExplainDetail], withRowType = false, - Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC)) + Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC), + withQueryBlockAlias = false) } /** @@ -313,7 +314,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) query, extraDetails.toArray, withRowType = false, - Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC)) + Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC), + withQueryBlockAlias = false) } /** @@ -406,7 +408,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) extraDetails.toArray, withRowType = false, Array(PlanKind.AST, PlanKind.OPT_REL, PlanKind.OPT_EXEC), - () => Unit) + () => Unit, + withQueryBlockAlias = false) } /** Verify the AST (abstract syntax tree). */ @@ -416,7 +419,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) Array.empty[ExplainDetail], withRowType = false, Array(PlanKind.AST), - () => Unit) + () => Unit, + withQueryBlockAlias = false) } /** Verify the AST (abstract syntax tree). The plans will contain the extra [[ExplainDetail]]s. */ @@ -426,7 +430,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) extraDetails.toArray, withRowType = false, Array(PlanKind.AST), - () => Unit) + () => Unit, + withQueryBlockAlias = false) } /** @@ -437,7 +442,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) query, Array.empty[ExplainDetail], withRowType = false, - Array(PlanKind.AST, PlanKind.OPT_REL)) + Array(PlanKind.AST, PlanKind.OPT_REL), + withQueryBlockAlias = false) } /** @@ -449,7 +455,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) query, extraDetails.toArray, withRowType = false, - Array(PlanKind.AST, PlanKind.OPT_REL)) + Array(PlanKind.AST, PlanKind.OPT_REL), + withQueryBlockAlias = false) } /** @@ -533,7 +540,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) Array.empty[ExplainDetail], withRowType = false, Array(PlanKind.AST, PlanKind.OPT_REL), - () => Unit) + () => Unit, + withQueryBlockAlias = false) } /** @@ -546,7 +554,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) extraDetails.toArray, withRowType = false, Array(PlanKind.AST, PlanKind.OPT_REL), - () => Unit) + () => Unit, + withQueryBlockAlias = false) } /** @@ -558,7 +567,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) query, Array.empty[ExplainDetail], withRowType = true, - Array(PlanKind.AST, PlanKind.OPT_REL)) + Array(PlanKind.AST, PlanKind.OPT_REL), + withQueryBlockAlias = false) } /** @@ -584,7 +594,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) Array.empty[ExplainDetail], withRowType = true, Array(PlanKind.AST, PlanKind.OPT_REL), - () => Unit) + () => Unit, + withQueryBlockAlias = false) } /** @@ -618,7 +629,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) query, Array.empty[ExplainDetail], withRowType = false, - Array(PlanKind.AST, PlanKind.OPT_EXEC)) + Array(PlanKind.AST, PlanKind.OPT_EXEC), + withQueryBlockAlias = false) } /** @@ -668,7 +680,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) Array.empty[ExplainDetail], withRowType = false, Array(PlanKind.AST, PlanKind.OPT_EXEC), - () => Unit) + () => Unit, + withQueryBlockAlias = false) } /** Verify the explain result for the given SELECT query. See more about [[Table#explain()]]. */ @@ -819,7 +832,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) query: String, extraDetails: Array[ExplainDetail], withRowType: Boolean, - expectedPlans: Array[PlanKind]): Unit = { + expectedPlans: Array[PlanKind], + withQueryBlockAlias: Boolean): Unit = { val table = getTableEnv.sqlQuery(query) val relNode = TableTestUtil.toRelNode(table) @@ -828,7 +842,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) extraDetails, withRowType, expectedPlans, - () => assertEqualsOrExpand("sql", query)) + () => assertEqualsOrExpand("sql", query), + withQueryBlockAlias) } /** @@ -855,7 +870,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) extraDetails, withRowType, expectedPlans, - () => assertEqualsOrExpand("sql", insert)) + () => assertEqualsOrExpand("sql", insert), + withQueryBlockAlias = false) } /** @@ -899,7 +915,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) extraDetails: Array[ExplainDetail], withRowType: Boolean, expectedPlans: Array[PlanKind], - assertSqlEqualsOrExpandFunc: () => Unit): Unit = { + assertSqlEqualsOrExpandFunc: () => Unit, + withQueryBlockAlias: Boolean): Unit = { val testStmtSet = stmtSet.asInstanceOf[StatementSetImpl[_]] val relNodes = testStmtSet.getOperations.map(getPlanner.translateToRel) @@ -915,7 +932,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) extraDetails, withRowType, expectedPlans, - assertSqlEqualsOrExpandFunc) + assertSqlEqualsOrExpandFunc, + withQueryBlockAlias) } /** @@ -932,13 +950,16 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) * @param assertSqlEqualsOrExpandFunc * the function to check whether the sql equals to the expected if the `relNodes` are translated * from sql + * @param withQueryBlockAlias + * whether the rel plans contains the query block alias, default is false */ - private def assertPlanEquals( + def assertPlanEquals( relNodes: Array[RelNode], extraDetails: Array[ExplainDetail], withRowType: Boolean, expectedPlans: Array[PlanKind], - assertSqlEqualsOrExpandFunc: () => Unit): Unit = { + assertSqlEqualsOrExpandFunc: () => Unit, + withQueryBlockAlias: Boolean = false): Unit = { // build ast plan val astBuilder = new StringBuilder @@ -946,8 +967,13 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) sink => astBuilder .append(System.lineSeparator) - .append(FlinkRelOptUtil - .toString(sink, SqlExplainLevel.EXPPLAN_ATTRIBUTES, withRowType = withRowType)) + .append( + FlinkRelOptUtil + .toString( + sink, + SqlExplainLevel.EXPPLAN_ATTRIBUTES, + withRowType = withRowType, + withQueryBlockAlias = withQueryBlockAlias)) } val astPlan = astBuilder.toString() @@ -1040,7 +1066,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean) str } - protected def assertEqualsOrExpand(tag: String, actual: String, expand: Boolean = true): Unit = { + def assertEqualsOrExpand(tag: String, actual: String, expand: Boolean = true): Unit = { val expected = s"$${$tag}" if (!expand) { diffRepository.assertEquals(test.name.getMethodName, tag, expected, actual)