diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterContext.java index 84f9f24e09deb8..d844b98371935d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterContext.java @@ -56,6 +56,7 @@ * runtime filter context used at post process and translation. */ public class RuntimeFilterContext { + public List prunedRF = Lists.newArrayList(); private final IdGenerator generator = RuntimeFilterId.createGenerator(); @@ -153,8 +154,11 @@ public void removeFilter(ExprId targetId, PhysicalHashJoin builderNode) { if (filters != null) { Iterator iter = filters.iterator(); while (iter.hasNext()) { - if (iter.next().getBuilderNode().equals(builderNode)) { + RuntimeFilter rf = iter.next(); + if (rf.getBuilderNode().equals(builderNode)) { + builderNode.getRuntimeFilters().remove(rf); iter.remove(); + prunedRF.add(rf); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java index 873e58c29fb2a8..c9cc43d0c29d9f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java @@ -27,7 +27,6 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalAssertNumRows; import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; -import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; import org.apache.doris.nereids.trees.plans.physical.PhysicalLimit; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; @@ -54,17 +53,6 @@ */ public class RuntimeFilterPruner extends PlanPostProcessor { - // ******************************* - // Physical plans - // ******************************* - @Override - public PhysicalHashAggregate visitPhysicalHashAggregate( - PhysicalHashAggregate agg, CascadesContext context) { - agg.child().accept(this, context); - context.getRuntimeFilterContext().addEffectiveSrcNode(agg); - return agg; - } - @Override public PhysicalQuickSort visitPhysicalQuickSort(PhysicalQuickSort sort, CascadesContext context) { sort.child().accept(this, context); @@ -165,7 +153,9 @@ public PhysicalAssertNumRows visitPhysicalAssertNumRows(PhysicalAssertNumRows buildNdvInProbeRange * (1 + ColumnStatistic.STATS_ERROR); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java index 80d33e7c85f7fb..b4a261847d83db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java +++ b/fe/fe-core/src/main/java/org/apache/doris/statistics/ColumnStatistic.java @@ -38,6 +38,8 @@ public class ColumnStatistic { + public static final double STATS_ERROR = 0.1D; + public static final StatsType NDV = StatsType.NDV; public static final StatsType AVG_SIZE = StatsType.AVG_SIZE; public static final StatsType MAX_SIZE = StatsType.MAX_SIZE; diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf10.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf10.groovy new file mode 100644 index 00000000000000..0e627ed78b961a --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf10.groovy @@ -0,0 +1,88 @@ +/* + * 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. + */ + +suite("rf10") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' +sql 'set be_number_for_test=3' + String query = """ + explain physical plan + select + c_custkey, + c_name, + sum(l_extendedprice * (1 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment + from + customer, + orders, + lineitem, + nation + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1993-10-01' + and o_orderdate < date '1993-10-01' + interval '3' month + and l_returnflag = 'R' + and c_nationkey = n_nationkey + group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment + order by + revenue desc + limit 20; + """ + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + // prune 1 RF + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h10 before prune:\n" + plan1) + log.info("tcph_sf1000 h10 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + + assertEquals(3, count1) + assertEquals(2, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf11.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf11.groovy new file mode 100644 index 00000000000000..3e38c5fd9577af --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf11.groovy @@ -0,0 +1,84 @@ +/* + * 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. + */ + +suite("rf11") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + sql 'set parallel_pipeline_task_num=8' + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' +sql 'set be_number_for_test=3' + + + + def String query = """ + explain physical plan + select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'GERMANY' + group by + ps_partkey having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.000002 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'GERMANY' + ) + order by + value desc; + """ + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + int count2 = getRuntimeFilterCountFromPlan(plan2) + + log.info("tcph_sf1000 h11 before prune:\n" + plan1) + log.info("tcph_sf1000 h11 after prune:\n" + plan2) + + assertEquals(4, count1) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf12.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf12.groovy new file mode 100644 index 00000000000000..10fa1cafd62fe3 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf12.groovy @@ -0,0 +1,86 @@ +/* + * 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. + */ + +suite("rf12") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + sql 'set parallel_pipeline_task_num=8' + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' +sql 'set be_number_for_test=3' + + def query = """ + explain physical plan + select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count + from + orders, + lineitem + where + o_orderkey = l_orderkey + and l_shipmode in ('MAIL', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1994-01-01' + and l_receiptdate < date '1994-01-01' + interval '1' year + group by + l_shipmode + order by + l_shipmode; + """ + + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h12 before prune:\n" + plan1) + log.info("tcph_sf1000 h12 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) + assertEquals(1, count1) + + +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf13.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf13.groovy new file mode 100644 index 00000000000000..c384aad2462978 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf13.groovy @@ -0,0 +1,79 @@ +/* + * 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. + */ + +suite("rf13") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + def query = """ + explain physical plan + select + c_count, + count(*) as custdist + from + ( + select + c_custkey, + count(o_orderkey) as c_count + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%special%requests%' + group by + c_custkey + ) as c_orders + group by + c_count + order by + custdist desc, + c_count desc; + """ + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h13 before prune:\n" + plan1) + log.info("tcph_sf1000 h13 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(0, count2) + assertEquals(1, count1) + +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf14.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf14.groovy new file mode 100644 index 00000000000000..873ca3f905b10a --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf14.groovy @@ -0,0 +1,68 @@ +/* + * 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. + */ + +suite("rf14") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' +sql 'set be_number_for_test=3' + def query = """ + explain physical plan + select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue + from + lineitem, + part + where + l_partkey = p_partkey + and l_shipdate >= date '1995-09-01' + and l_shipdate < date '1995-09-01' + interval '1' month; + """ + + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h14 before prune:\n" + plan1) + log.info("tcph_sf1000 h14 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf15.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf15.groovy new file mode 100644 index 00000000000000..8c795b19cc21a2 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf15.groovy @@ -0,0 +1,79 @@ +/* + * 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. + */ + +suite("rf15") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + def query = """ + explain physical plan + select + s_suppkey, + s_name, + s_address, + s_phone, + total_revenue + from + supplier, + revenue0 + where + s_suppkey = supplier_no + and total_revenue = ( + select + max(total_revenue) + from + revenue0 + ) + order by + s_suppkey; + """ + + + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h15 before prune:\n" + plan1) + log.info("tcph_sf1000 h15 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf16.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf16.groovy new file mode 100644 index 00000000000000..896c66c2232afa --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf16.groovy @@ -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. + */ + +suite("rf16") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + def query = """ + explain physical plan + select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt + from + partsupp, + part + where + p_partkey = ps_partkey + and p_brand <> 'Brand#45' + and p_type not like 'MEDIUM POLISHED%' + and p_size in (49, 14, 23, 45, 19, 3, 36, 9) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' + ) + group by + p_brand, + p_type, + p_size + order by + supplier_cnt desc, + p_brand, + p_type, + p_size; + """ + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h16 before prune:\n" + plan1) + log.info("tcph_sf1000 h16 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf17.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf17.groovy new file mode 100644 index 00000000000000..cc2b4786d4cdb0 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf17.groovy @@ -0,0 +1,74 @@ +/* + * 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. + */ + +suite("rf17") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + def query = """ + explain physical plan + select + sum(l_extendedprice) / 7.0 as avg_yearly + from + lineitem, + part + where + p_partkey = l_partkey + and p_brand = 'Brand#23' + and p_container = 'MED BOX' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey + ); + """ + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h17 before prune:\n" + plan1) + log.info("tcph_sf1000 h17 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf18.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf18.groovy new file mode 100644 index 00000000000000..07c38e83967253 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf18.groovy @@ -0,0 +1,90 @@ +/* + * 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. + */ + +suite("rf18") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + def query = """ + explain physical plan + select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) + from + customer, + orders, + lineitem + where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey having + sum(l_quantity) > 300 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey + group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice + order by + o_totalprice desc, + o_orderdate + limit 100; + """ + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h18 before prune:\n" + plan1) + log.info("tcph_sf1000 h18 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf19.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf19.groovy new file mode 100644 index 00000000000000..5ca7434c24be1d --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf19.groovy @@ -0,0 +1,94 @@ +/* + * 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. + */ + +suite("rf19") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + def query = """ + explain physical plan + select + sum(l_extendedprice* (1 - l_discount)) as revenue + from + lineitem, + part + where + ( + p_partkey = l_partkey + and p_brand = 'Brand#12' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 1 and l_quantity <= 1 + 10 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#23' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 10 and l_quantity <= 10 + 10 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#34' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 20 and l_quantity <= 20 + 10 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); + + """ + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h19 before prune:\n" + plan1) + log.info("tcph_sf1000 h19 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf2.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf2.groovy new file mode 100644 index 00000000000000..0a53ee87955f77 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf2.groovy @@ -0,0 +1,102 @@ +/* + * 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. + */ + +suite("rf2") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + + sql 'set be_number_for_test=3' + + String query = """ + explain physical plan + select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment + from + part, + supplier, + partsupp, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 15 + and p_type like '%BRASS' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'EUROPE' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'EUROPE' + ) + order by + s_acctbal desc, + n_name, + s_name, + p_partkey + limit 100; + """ + sql "set enable_runtime_filter_prune=false" + String plan1 = sql("${query}") + int count1 = getRuntimeFilterCountFromPlan(plan1) + + sql "set enable_runtime_filter_prune=true" + String plan2 = sql("${query}") + + log.info("tcph_sf1000 h2 before prune:\n" + plan1) + log.info("tcph_sf1000 h2 after prune:\n" + plan2) + int count2 = getRuntimeFilterCountFromPlan(plan2) + + assertEquals(4, count1) + assertEquals(4, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf20.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf20.groovy new file mode 100644 index 00000000000000..7fd96509db9478 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf20.groovy @@ -0,0 +1,95 @@ +/* + * 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. + */ + +suite("rf20") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + def query = """ + explain physical plan + select + s_name, + s_address + from + supplier, + nation + where + s_suppkey in ( + select + ps_suppkey + from + partsupp + where + ps_partkey in ( + select + p_partkey + from + part + where + p_name like 'forest%' + ) + and ps_availqty > ( + select + 0.5 * sum(l_quantity) + from + lineitem + where + l_partkey = ps_partkey + and l_suppkey = ps_suppkey + and l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + ) + ) + and s_nationkey = n_nationkey + and n_name = 'CANADA' + order by + s_name; + """ + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h20 before prune:\n" + plan1) + log.info("tcph_sf1000 h20 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf21.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf21.groovy new file mode 100644 index 00000000000000..1fa097075247cd --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf21.groovy @@ -0,0 +1,97 @@ +/* + * 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. + */ + +suite("rf21") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + def query = """ + explain physical plan + select + s_name, + count(*) as numwait + from + supplier, + lineitem l1, + orders, + nation + where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'SAUDI ARABIA' + group by + s_name + order by + numwait desc, + s_name + limit 100; + """ + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h21 before prune:\n" + plan1) + log.info("tcph_sf1000 h21 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf22.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf22.groovy new file mode 100644 index 00000000000000..30c31adc342562 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf22.groovy @@ -0,0 +1,94 @@ +/* + * 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. + */ + +suite("q22") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + def query = """ + explain physical plan + select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal + from + ( + select + substring(c_phone, 1, 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone, 1, 2) in + ('13', '31', '23', '29', '30', '18', '17') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00 + and substring(c_phone, 1, 2) in + ('13', '31', '23', '29', '30', '18', '17') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale + group by + cntrycode + order by + cntrycode; + """ + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h22 before prune:\n" + plan1) + log.info("tcph_sf1000 h22 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf3.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf3.groovy new file mode 100644 index 00000000000000..882c6c5e1e71cb --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf3.groovy @@ -0,0 +1,83 @@ +/* + * 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. + */ + +suite("rf3") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + // db = "tpch" + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + + String query = """ + explain physical plan + select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority + from + customer, + orders, + lineitem + where + c_mktsegment = 'BUILDING' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-15' + and l_shipdate > date '1995-03-15' + group by + l_orderkey, + o_orderdate, + o_shippriority + order by + revenue desc, + o_orderdate + limit 10; + """ + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h3 before prune:\n" + plan1) + log.info("tcph_sf1000 h3 before prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf4.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf4.groovy new file mode 100644 index 00000000000000..d947f4afb0226c --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf4.groovy @@ -0,0 +1,79 @@ +/* + * 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. + */ + +suite("rf4") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set be_number_for_test=3' + sql 'set parallel_pipeline_task_num=8' + + + + + String query = """ + explain physical plan + select + o_orderpriority, + count(*) as order_count + from + orders + where + o_orderdate >= date '1993-07-01' + and o_orderdate < date '1993-07-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) + group by + o_orderpriority + order by + o_orderpriority; + """ + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h4 before prune:\n" + plan1) + log.info("tcph_sf1000 h4 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf5.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf5.groovy new file mode 100644 index 00000000000000..017d1ce2984a18 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf5.groovy @@ -0,0 +1,80 @@ +/* + * 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. + */ + +suite("rf5") { + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + sql 'set be_number_for_test=3' + + sql 'set enable_runtime_filter_prune=false' + + String query = """ + explain physical plan + select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue + from + customer, + orders, + lineitem, + supplier, + nation, + region + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'ASIA' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '1' year + group by + n_name + order by + revenue desc; + """ + String plan1 = sql "${query}" + + + sql 'set enable_runtime_filter_prune=true' + + String plan2 = sql "${query}" + log.info("tcph_sf1000 h5 before prune:\n" + plan1) + log.info("tcph_sf1000 h5 after prune:\n" + plan2) + + assertEquals(6, getRuntimeFilterCountFromPlan(plan1)) + assertEquals(4, getRuntimeFilterCountFromPlan(plan2)) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf7.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf7.groovy new file mode 100644 index 00000000000000..5e1204fc2901d9 --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf7.groovy @@ -0,0 +1,99 @@ +/* + * 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. + */ + +suite("rf7") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + String query = """ + explain physical plan + select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue + from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY') + or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE') + ) + and l_shipdate between date '1995-01-01' and date '1996-12-31' + ) as shipping + group by + supp_nation, + cust_nation, + l_year + order by + supp_nation, + cust_nation, + l_year; + """ + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h7 before prune:\n" + plan1) + log.info("tcph_sf1000 h7 after prune:\n" + plan2) + + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(5, count2, "after prune") + assertEquals(5, count1, "before prune") + +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf8.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf8.groovy new file mode 100644 index 00000000000000..565d4cbb40242e --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf8.groovy @@ -0,0 +1,97 @@ +/* + * 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. + */ + +suite("rf8") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + String query = """ + explain physical plan + select + o_year, + sum(case + when nation = 'BRAZIL' then volume + else 0 + end) / sum(volume) as mkt_share + from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'AMERICA' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'ECONOMY ANODIZED STEEL' + ) as all_nations + group by + o_year + order by + o_year; + """ + + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + // prune 1 RF + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + log.info "before prune, rf count = " + count1 + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h8 before prune:\n" + plan1) + log.info("tcph_sf1000 h8 after prune:\n" + plan2) + + assertEquals(7, count1) + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(count1 - 1, count2) +} diff --git a/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf9.groovy b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf9.groovy new file mode 100644 index 00000000000000..be978a6017846b --- /dev/null +++ b/regression-test/suites/nereids_tpch_shape_sf1000_p0/rf_prune/rf9.groovy @@ -0,0 +1,91 @@ +/* + * 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. + */ + +suite("rf9") { + String db = context.config.getDbNameByFile(new File(context.file.parent)) + sql "use ${db}" + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql "set runtime_filter_mode='GLOBAL'" + + sql 'set exec_mem_limit=21G' + sql 'SET enable_pipeline_engine = true' + sql 'set parallel_pipeline_task_num=8' + + + +sql 'set be_number_for_test=3' + + String query = """ + explain physical plan + select + nation, + o_year, + sum(amount) as sum_profit + from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%green%' + ) as profit + group by + nation, + o_year + order by + nation, + o_year desc; + """ + def getRuntimeFilterCountFromPlan = { plan -> { + int count = 0 + plan.eachMatch("RF\\d+\\[") { + ch -> count ++ + } + return count + }} + // prune 4 RF + sql "set enable_runtime_filter_prune=false" + String plan1 = sql "${query}" + int count1 = getRuntimeFilterCountFromPlan(plan1) + sql "set enable_runtime_filter_prune=true" + String plan2 = sql "${query}" + + log.info("tcph_sf1000 h9 before prune:\n" + plan1) + log.info("tcph_sf1000 h9 after prune:\n" + plan2) + + assertEquals(6, count1) + assertEquals(6, getRuntimeFilterCountFromPlan(plan1)) + int count2 = getRuntimeFilterCountFromPlan(plan2) + assertEquals(2, count2) +}