godfreyhe commented on code in PR #20303: URL: https://github.com/apache/flink/pull/20303#discussion_r934425287
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java: ########## @@ -0,0 +1,552 @@ +/* + * 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.rules.logical; + +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.plan.RelRule; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.calcite.rel.rules.CoreRules; +import org.apache.calcite.rel.rules.FilterMultiJoinMergeRule; +import org.apache.calcite.rel.rules.MultiJoin; +import org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule; +import org.apache.calcite.rel.rules.TransformationRule; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexUtil; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.ImmutableIntList; +import org.apache.calcite.util.Pair; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Flink Planner rule to flatten a tree of {@link LogicalJoin}s into a single {@link MultiJoin} with + * N inputs. + * + * <p>An input is not flattened if the input is a null generating input in an outer join, i.e., + * either input in a full outer join, the right hand side of a left outer join, or the left hand + * side of a right outer join. + * + * <p>Join conditions are also pulled up from the inputs into the topmost {@link MultiJoin}, unless + * the input corresponds to a null generating input in an outer join, + * + * <p>Outer join information is also stored in the {@link MultiJoin}. A boolean flag indicates if + * the join is a full outer join, and in the case of left and right outer joins, the join type and + * outer join conditions are stored in arrays in the {@link MultiJoin}. This outer join information + * is associated with the null generating input in the outer join. So, in the case of a left outer + * join between A and B, the information is associated with B, not A. + * + * <p>Here are examples of the {@link MultiJoin}s constructed after this rule has been applied on + * following join trees. + * + * <ul> + * <li>A JOIN B → MJ(A, B) + * <li>A JOIN B JOIN C → MJ(A, B, C) + * <li>A LEFT JOIN B → MJ(A, B), left outer join on input#1 + * <li>A RIGHT JOIN B → MJ(A, B), right outer join on input#0 + * <li>A FULL JOIN B → MJ[full](A, B) + * <li>A LEFT JOIN (B JOIN C) → MJ(A, MJ(B, C))), left outer join on input#1 in the outermost + * MultiJoin + * <li>(A JOIN B) LEFT JOIN C → MJ(A, B, C), left outer join on input#2 + * <li>(A LEFT JOIN B) JOIN C → MJ(MJ(A, B), C), left outer join on input#1 of the inner + * MultiJoin TODO + * <li>A LEFT JOIN (B FULL JOIN C) → MJ(A, MJ[full](B, C)), left outer join on input#1 in the + * outermost MultiJoin + * <li>(A LEFT JOIN B) FULL JOIN (C RIGHT JOIN D) → MJ[full](MJ(A, B), MJ(C, D)), left outer + * join on input #1 in the first inner MultiJoin and right outer join on input#0 in the second + * inner MultiJoin + * </ul> + * + * <p>The constructor is parameterized to allow any sub-class of {@link Join}, not just {@link + * LogicalJoin}. + * + * @see FilterMultiJoinMergeRule + * @see ProjectMultiJoinMergeRule + * @see CoreRules#JOIN_TO_MULTI_JOIN + */ +public class FlinkJoinToMultiJoinRule extends RelRule<FlinkJoinToMultiJoinRule.Config> Review Comment: add some comments about the relation between FlinkJoinToMultiJoinRule and JoinToMultiJoinRule ########## flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRuleTest.scala: ########## @@ -49,6 +49,36 @@ class FlinkJoinToMultiJoinRuleTest extends TableTestBase { util.addTableSource[(Int, Long)]("T3", 'e, 'f) } + @Test Review Comment: please add more complex tests:, such as (J means Join, LJ means LeftJoin, RJ means RightJoin) 1. A J B LJ C J D LJ E 2. A J B RJ C J D RJ E 3. A RJ B RJ C 4. A RJ B J C 5. A FJ B J C 6. A FJ B LJ C 7. A FJ B RJ C 8. A LJ B J C 9. ... ########## flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/MultiJoinOptimizeTest.scala: ########## @@ -0,0 +1,170 @@ +/* + * 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.rules.logical + +import org.apache.flink.table.catalog.ObjectPath +import org.apache.flink.table.catalog.stats.CatalogTableStatistics +import org.apache.flink.table.planner.plan.optimize.program._ +import org.apache.flink.table.planner.utils.{TableConfigUtils, TableTestBase} + +import org.apache.calcite.plan.hep.HepMatchOrder +import org.apache.calcite.rel.rules.CoreRules +import org.apache.calcite.tools.RuleSets +import org.junit.{Before, Test} + +/** + * Plan tests for join order using rule [[org.apache.calcite.rel.rules.LoptOptimizeJoinRule]] after + * joins converts to multi join set by + * [[org.apache.flink.table.planner.plan.rules.logical.FlinkJoinToMultiJoinRule]]. + */ +class MultiJoinOptimizeTest extends TableTestBase { Review Comment: If we want to test join reorder, please add test cases into JoinReorderTestBase ########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/FlinkJoinToMultiJoinRule.java: ########## @@ -0,0 +1,552 @@ +/* + * 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.rules.logical; + +import org.apache.calcite.plan.RelOptRuleCall; +import org.apache.calcite.plan.RelOptUtil; +import org.apache.calcite.plan.RelRule; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.logical.LogicalJoin; +import org.apache.calcite.rel.rules.CoreRules; +import org.apache.calcite.rel.rules.FilterMultiJoinMergeRule; +import org.apache.calcite.rel.rules.MultiJoin; +import org.apache.calcite.rel.rules.ProjectMultiJoinMergeRule; +import org.apache.calcite.rel.rules.TransformationRule; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexBuilder; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexUtil; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.ImmutableIntList; +import org.apache.calcite.util.Pair; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Flink Planner rule to flatten a tree of {@link LogicalJoin}s into a single {@link MultiJoin} with + * N inputs. + * + * <p>An input is not flattened if the input is a null generating input in an outer join, i.e., + * either input in a full outer join, the right hand side of a left outer join, or the left hand + * side of a right outer join. + * + * <p>Join conditions are also pulled up from the inputs into the topmost {@link MultiJoin}, unless + * the input corresponds to a null generating input in an outer join, + * + * <p>Outer join information is also stored in the {@link MultiJoin}. A boolean flag indicates if + * the join is a full outer join, and in the case of left and right outer joins, the join type and + * outer join conditions are stored in arrays in the {@link MultiJoin}. This outer join information + * is associated with the null generating input in the outer join. So, in the case of a left outer + * join between A and B, the information is associated with B, not A. + * + * <p>Here are examples of the {@link MultiJoin}s constructed after this rule has been applied on + * following join trees. + * + * <ul> + * <li>A JOIN B → MJ(A, B) + * <li>A JOIN B JOIN C → MJ(A, B, C) + * <li>A LEFT JOIN B → MJ(A, B), left outer join on input#1 + * <li>A RIGHT JOIN B → MJ(A, B), right outer join on input#0 + * <li>A FULL JOIN B → MJ[full](A, B) + * <li>A LEFT JOIN (B JOIN C) → MJ(A, MJ(B, C))), left outer join on input#1 in the outermost + * MultiJoin + * <li>(A JOIN B) LEFT JOIN C → MJ(A, B, C), left outer join on input#2 + * <li>(A LEFT JOIN B) JOIN C → MJ(MJ(A, B), C), left outer join on input#1 of the inner Review Comment: This line can be updated -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org