swuferhong commented on code in PR #21489: URL: https://github.com/apache/flink/pull/21489#discussion_r1063260676
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/FlinkDynamicPartitionPruningProgram.java: ########## @@ -0,0 +1,151 @@ +/* + * 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.program; + +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalDynamicFilteringTableSourceScan; +import org.apache.flink.table.planner.plan.nodes.physical.batch.BatchPhysicalTableSourceScan; +import org.apache.flink.table.planner.plan.utils.DefaultRelShuttle; +import org.apache.flink.table.planner.utils.DynamicPartitionPruningUtils; +import org.apache.flink.table.planner.utils.ShortcutUtils; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.JoinInfo; +import org.apache.calcite.rel.core.JoinRelType; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Planner program that tries to do partition prune in the execution phase, which can translate a + * {@link BatchPhysicalTableSourceScan} to a {@link BatchPhysicalDynamicFilteringTableSourceScan} + * whose source is a partition source. The {@link + * OptimizerConfigOptions#TABLE_OPTIMIZER_DYNAMIC_FILTERING_ENABLED} need to be true. + * + * <p>Suppose we have the original physical plan: + * + * <pre>{@code + * LogicalProject(...) + * HashJoin(joinType=[InnerJoin], where=[=(fact_partition_key, dim_key)], select=[xxx]) + * * :- TableSourceScan(table=[[fact]], fields=[xxx, fact_partition_key],) # Is a partition table. + * * +- Exchange(distribution=[broadcast]) + * * +- Calc(select=[xxx], where=[<(xxx, xxx)]) # Need have an arbitrary filter condition. + * * +- TableSourceScan(table=[[dim, filter=[]]], fields=[xxx, dim_key]) + * }</pre> + * + * <p>This physical plan will be rewritten to: + * + * <pre>{@code + * HashJoin(joinType=[InnerJoin], where=[=(fact_partition_key, dim_key)], select=[xxx]) + * :- DynamicFilteringTableSourceScan(table=[[fact]], fields=[xxx, fact_partition_key]) # Is a partition table. + * : +- DynamicFilteringDataCollector(fields=[dim_key]) + * : +- Calc(select=[xxx], where=[<(xxx, xxx)]) + * : +- TableSourceScan(table=[[dim, filter=[]]], fields=[xxx, dim_key]) + * +- Exchange(distribution=[broadcast]) + * +- Calc(select=[xxx], where=[<(xxx, xxx)]) # Need have an arbitrary filter condition. + * +- TableSourceScan(table=[[dim, filter=[]]], fields=[xxx, dim_key]) + * }</pre> + * + * <p>We use a {@link FlinkOptimizeProgram} instead of a {@link org.apache.calcite.plan.RelRule} to + * realize dynamic partition pruning because the {@link org.apache.calcite.plan.hep.HepPlanner} in + * Flink doesn't support matching a simple join, and replacing one node on one side of the join + * node. After that, rebuilding this join node. This is a defect of the existing optimizer, and it's + * matching pattern need to be simpler. Only then can we use {@link org.apache.calcite.plan.RelRule} + * to achieve dpp. Review Comment: > * NOTE: We use a {@link FlinkOptimizeProgram} instead of a {@link > * org.apache.calcite.plan.RelRule} here because the {@link org.apache.calcite.plan.hep.HepPlanner} > * doesn't support matching a partially determined pattern or dynamically replacing the inputs of > * matched nodes. Once we improve HepPlanner, this class can be converted to RelRule. Done! -- 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