[Test](Nereids) Add SSB dataset to test materialized view rewrite (#30528)
* [Test](Nereids) Add SSB dataset to test materialized view rewrite * rollback irrelevant code * fix sort slot 0
This commit is contained in:
@ -72,7 +72,7 @@ public class MTMVCache {
|
||||
? (Plan) ((LogicalResultSink) mvRewrittenPlan).child() : mvRewrittenPlan;
|
||||
// use rewritten plan output expression currently, if expression rewrite fail,
|
||||
// consider to use the analyzed plan for output expressions only
|
||||
List<NamedExpression> mvOutputExpressions = mvPlan.getExpressions().stream()
|
||||
List<NamedExpression> mvOutputExpressions = mvPlan.getOutput().stream()
|
||||
.map(NamedExpression.class::cast)
|
||||
.collect(Collectors.toList());
|
||||
return new MTMVCache(mvPlan, mvOutputExpressions);
|
||||
|
||||
@ -42,10 +42,13 @@ import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewri
|
||||
import org.apache.doris.nereids.trees.plans.Plan;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
|
||||
import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer;
|
||||
import org.apache.doris.nereids.types.BigIntType;
|
||||
import org.apache.doris.nereids.util.ExpressionUtils;
|
||||
|
||||
import com.google.common.collect.ArrayListMultimap;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Multimap;
|
||||
import com.google.common.collect.Sets;
|
||||
|
||||
@ -118,7 +121,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
|
||||
// Firstly,if group by expression between query and view is equals, try to rewrite expression directly
|
||||
Plan queryTopPlan = queryTopPlanAndAggPair.key();
|
||||
if (isGroupByEquals(queryTopPlanAndAggPair, viewTopPlanAndAggPair, viewToQuerySlotMapping)) {
|
||||
List<Expression> rewrittenQueryExpressions = rewriteExpression(queryTopPlan.getExpressions(),
|
||||
List<Expression> rewrittenQueryExpressions = rewriteExpression(queryTopPlan.getOutput(),
|
||||
queryTopPlan,
|
||||
materializationContext.getMvExprToMvScanExprMapping(),
|
||||
viewToQuerySlotMapping,
|
||||
@ -135,7 +138,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
|
||||
Pair.of("Can not rewrite expression when no roll up",
|
||||
String.format("expressionToWrite = %s,\n mvExprToMvScanExprMapping = %s,\n"
|
||||
+ "viewToQuerySlotMapping = %s",
|
||||
queryTopPlan.getExpressions(),
|
||||
queryTopPlan.getOutput(),
|
||||
materializationContext.getMvExprToMvScanExprMapping(),
|
||||
viewToQuerySlotMapping)));
|
||||
}
|
||||
@ -161,7 +164,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
|
||||
// try to rewrite, contains both roll up aggregate functions and aggregate group expression
|
||||
List<NamedExpression> finalOutputExpressions = new ArrayList<>();
|
||||
List<Expression> finalGroupExpressions = new ArrayList<>();
|
||||
List<? extends Expression> queryExpressions = queryTopPlan.getExpressions();
|
||||
List<? extends Expression> queryExpressions = queryTopPlan.getOutput();
|
||||
// permute the mv expr mapping to query based
|
||||
Map<Expression, Expression> mvExprToMvScanExprQueryBased =
|
||||
materializationContext.getMvExprToMvScanExprMapping().keyPermute(viewToQuerySlotMapping)
|
||||
@ -231,13 +234,8 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
finalOutputExpressions = finalOutputExpressions.stream()
|
||||
.map(expr -> {
|
||||
ExprId exprId = expr.getExprId();
|
||||
if (projectOutPutExprIdMap.containsKey(exprId)) {
|
||||
return projectOutPutExprIdMap.get(exprId);
|
||||
}
|
||||
return expr;
|
||||
})
|
||||
.map(expr -> projectOutPutExprIdMap.containsKey(expr.getExprId())
|
||||
? projectOutPutExprIdMap.get(expr.getExprId()) : expr)
|
||||
.collect(Collectors.toList());
|
||||
return new LogicalAggregate(finalGroupExpressions, finalOutputExpressions, mvProject);
|
||||
}
|
||||
@ -327,24 +325,29 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
|
||||
|
||||
private Pair<Set<? extends Expression>, Set<? extends Expression>> topPlanSplitToGroupAndFunction(
|
||||
Pair<Plan, LogicalAggregate<Plan>> topPlanAndAggPair) {
|
||||
LogicalAggregate<Plan> queryAggregate = topPlanAndAggPair.value();
|
||||
Set<Expression> queryAggGroupSet = new HashSet<>(queryAggregate.getGroupByExpressions());
|
||||
LogicalAggregate<Plan> bottomQueryAggregate = topPlanAndAggPair.value();
|
||||
Set<Expression> groupByExpressionSet = new HashSet<>(bottomQueryAggregate.getGroupByExpressions());
|
||||
// when query is bitmap_count(bitmap_union), the plan is as following:
|
||||
// project(bitmap_count()#1)
|
||||
// aggregate(bitmap_union()#2)
|
||||
// we should use exprId which query top plan used to decide the query top plan is use the
|
||||
// bottom agg function or not
|
||||
Set<ExprId> queryAggFunctionSet = queryAggregate.getOutputExpressions().stream()
|
||||
.filter(expr -> !queryAggGroupSet.contains(expr))
|
||||
Set<ExprId> bottomAggregateFunctionExprIdSet = bottomQueryAggregate.getOutput().stream()
|
||||
.filter(expr -> !groupByExpressionSet.contains(expr))
|
||||
.map(NamedExpression::getExprId)
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
Plan queryTopPlan = topPlanAndAggPair.key();
|
||||
Set<Expression> topGroupByExpressions = new HashSet<>();
|
||||
Set<Expression> topFunctionExpressions = new HashSet<>();
|
||||
queryTopPlan.getExpressions().forEach(expression -> {
|
||||
if (expression.anyMatch(expr -> expr instanceof NamedExpression
|
||||
&& queryAggFunctionSet.contains(((NamedExpression) expr).getExprId()))) {
|
||||
queryTopPlan.getOutput().forEach(expression -> {
|
||||
ExpressionLineageReplacer.ExpressionReplaceContext replaceContext =
|
||||
new ExpressionLineageReplacer.ExpressionReplaceContext(ImmutableList.of(expression),
|
||||
ImmutableSet.of(), ImmutableSet.of());
|
||||
queryTopPlan.accept(ExpressionLineageReplacer.INSTANCE, replaceContext);
|
||||
if (!Sets.intersection(bottomAggregateFunctionExprIdSet,
|
||||
replaceContext.getExprIdExpressionMap().keySet()).isEmpty()) {
|
||||
// if query top plan expression use any aggregate function, then consider it is aggregate function
|
||||
topFunctionExpressions.add(expression);
|
||||
} else {
|
||||
topGroupByExpressions.add(expression);
|
||||
@ -378,14 +381,13 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
|
||||
HyperGraph hyperGraph = structInfo.getHyperGraph();
|
||||
for (AbstractNode node : hyperGraph.getNodes()) {
|
||||
StructInfoNode structInfoNode = (StructInfoNode) node;
|
||||
if (!structInfoNode.getPlan().accept(StructInfo.JOIN_PATTERN_CHECKER,
|
||||
SUPPORTED_JOIN_TYPE_SET)) {
|
||||
if (!structInfoNode.getPlan().accept(StructInfo.JOIN_PATTERN_CHECKER, SUPPORTED_JOIN_TYPE_SET)) {
|
||||
return false;
|
||||
}
|
||||
for (JoinEdge edge : hyperGraph.getJoinEdges()) {
|
||||
if (!edge.getJoin().accept(StructInfo.JOIN_PATTERN_CHECKER, SUPPORTED_JOIN_TYPE_SET)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
for (JoinEdge edge : hyperGraph.getJoinEdges()) {
|
||||
if (!edge.getJoin().accept(StructInfo.JOIN_PATTERN_CHECKER, SUPPORTED_JOIN_TYPE_SET)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
|
||||
@ -29,9 +29,6 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression;
|
||||
import org.apache.doris.nereids.trees.plans.Plan;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@ -41,9 +38,6 @@ import java.util.stream.Collectors;
|
||||
*/
|
||||
public abstract class AbstractMaterializedViewJoinRule extends AbstractMaterializedViewRule {
|
||||
|
||||
protected final String currentClassName = this.getClass().getSimpleName();
|
||||
private final Logger logger = LogManager.getLogger(this.getClass());
|
||||
|
||||
@Override
|
||||
protected Plan rewriteQueryByView(MatchMode matchMode,
|
||||
StructInfo queryStructInfo,
|
||||
|
||||
@ -195,6 +195,10 @@ public class MaterializedViewUtils {
|
||||
@Override
|
||||
public Void visitLogicalJoin(LogicalJoin<? extends Plan, ? extends Plan> join,
|
||||
IncrementCheckerContext context) {
|
||||
if (join.isMarkJoin()) {
|
||||
context.setPctPossible(false);
|
||||
return null;
|
||||
}
|
||||
Plan left = join.child(0);
|
||||
Set<Column> leftColumnSet = left.getOutputSet().stream()
|
||||
.filter(slot -> slot instanceof SlotReference
|
||||
|
||||
@ -37,6 +37,7 @@ import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation;
|
||||
import org.apache.doris.nereids.trees.plans.algebra.Filter;
|
||||
import org.apache.doris.nereids.trees.plans.algebra.Join;
|
||||
import org.apache.doris.nereids.trees.plans.algebra.Project;
|
||||
import org.apache.doris.nereids.trees.plans.algebra.Sort;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
|
||||
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
|
||||
@ -457,6 +458,7 @@ public class StructInfo {
|
||||
&& !(plan instanceof Project)
|
||||
&& !(plan instanceof CatalogRelation)
|
||||
&& !(plan instanceof Join)
|
||||
&& !(plan instanceof Sort)
|
||||
&& !(plan instanceof LogicalAggregate && !((LogicalAggregate) plan).getSourceRepeat()
|
||||
.isPresent())) {
|
||||
return false;
|
||||
@ -489,7 +491,7 @@ public class StructInfo {
|
||||
super.visit(aggregate, context);
|
||||
return true;
|
||||
}
|
||||
if (plan instanceof LogicalProject || plan instanceof LogicalFilter) {
|
||||
if (plan instanceof Project || plan instanceof Filter || plan instanceof Sort) {
|
||||
super.visit(plan, context);
|
||||
return true;
|
||||
}
|
||||
|
||||
3955
regression-test/data/nereids_rules_p0/mv/ssb/mv_ssb_test.out
Normal file
3955
regression-test/data/nereids_rules_p0/mv/ssb/mv_ssb_test.out
Normal file
File diff suppressed because it is too large
Load Diff
@ -388,9 +388,9 @@ suite("outer_join") {
|
||||
|
||||
|
||||
def mv3_2 = """
|
||||
select lineitem.L_LINENUMBER, t2.O_CUSTKEY, t2.O_ORDERSTATUS
|
||||
from lineitem
|
||||
left join
|
||||
select lineitem.L_LINENUMBER, t2.O_CUSTKEY, t2.O_ORDERSTATUS
|
||||
from lineitem
|
||||
left join
|
||||
(select * from orders where O_ORDERSTATUS = 'o') t2
|
||||
on lineitem.L_ORDERKEY = t2.O_ORDERKEY;
|
||||
"""
|
||||
|
||||
@ -0,0 +1,15 @@
|
||||
CREATE TABLE IF NOT EXISTS `customer` (
|
||||
`c_custkey` int(11) NOT NULL COMMENT "",
|
||||
`c_name` varchar(26) NOT NULL COMMENT "",
|
||||
`c_address` varchar(41) NOT NULL COMMENT "",
|
||||
`c_city` varchar(11) NOT NULL COMMENT "",
|
||||
`c_nation` varchar(16) NOT NULL COMMENT "",
|
||||
`c_region` varchar(13) NOT NULL COMMENT "",
|
||||
`c_phone` varchar(16) NOT NULL COMMENT "",
|
||||
`c_mktsegment` varchar(11) NOT NULL COMMENT ""
|
||||
)
|
||||
UNIQUE KEY (`c_custkey`)
|
||||
DISTRIBUTED BY HASH(`c_custkey`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
@ -0,0 +1 @@
|
||||
truncate table customer;
|
||||
@ -0,0 +1,24 @@
|
||||
CREATE TABLE IF NOT EXISTS `date` (
|
||||
`d_datekey` int(11) NOT NULL COMMENT "",
|
||||
`d_date` varchar(20) NOT NULL COMMENT "",
|
||||
`d_dayofweek` varchar(10) NOT NULL COMMENT "",
|
||||
`d_month` varchar(11) NOT NULL COMMENT "",
|
||||
`d_year` int(11) NOT NULL COMMENT "",
|
||||
`d_yearmonthnum` int(11) NOT NULL COMMENT "",
|
||||
`d_yearmonth` varchar(9) NOT NULL COMMENT "",
|
||||
`d_daynuminweek` int(11) NOT NULL COMMENT "",
|
||||
`d_daynuminmonth` int(11) NOT NULL COMMENT "",
|
||||
`d_daynuminyear` int(11) NOT NULL COMMENT "",
|
||||
`d_monthnuminyear` int(11) NOT NULL COMMENT "",
|
||||
`d_weeknuminyear` int(11) NOT NULL COMMENT "",
|
||||
`d_sellingseason` varchar(14) NOT NULL COMMENT "",
|
||||
`d_lastdayinweekfl` int(11) NOT NULL COMMENT "",
|
||||
`d_lastdayinmonthfl` int(11) NOT NULL COMMENT "",
|
||||
`d_holidayfl` int(11) NOT NULL COMMENT "",
|
||||
`d_weekdayfl` int(11) NOT NULL COMMENT ""
|
||||
)
|
||||
UNIQUE KEY (`d_datekey`)
|
||||
DISTRIBUTED BY HASH(`d_datekey`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
@ -0,0 +1 @@
|
||||
truncate table `date`;
|
||||
@ -0,0 +1,24 @@
|
||||
CREATE TABLE IF NOT EXISTS `lineorder` (
|
||||
`lo_orderkey` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_linenumber` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_custkey` int(11) NOT NULL COMMENT "",
|
||||
`lo_partkey` int(11) NOT NULL COMMENT "",
|
||||
`lo_suppkey` int(11) NOT NULL COMMENT "",
|
||||
`lo_orderdate` int(11) NOT NULL COMMENT "",
|
||||
`lo_orderpriority` varchar(16) NOT NULL COMMENT "",
|
||||
`lo_shippriority` int(11) NOT NULL COMMENT "",
|
||||
`lo_quantity` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_extendedprice` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_ordtotalprice` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_discount` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_revenue` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_supplycost` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_tax` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_commitdate` bigint(20) NOT NULL COMMENT "",
|
||||
`lo_shipmode` varchar(11) NOT NULL COMMENT ""
|
||||
)
|
||||
UNIQUE KEY (`lo_orderkey`, `lo_linenumber`)
|
||||
DISTRIBUTED BY HASH(`lo_orderkey`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
@ -0,0 +1 @@
|
||||
truncate table lineorder;
|
||||
@ -0,0 +1,45 @@
|
||||
CREATE TABLE IF NOT EXISTS `lineorder_flat` (
|
||||
`LO_ORDERDATE` date NOT NULL COMMENT "",
|
||||
`LO_ORDERKEY` int(11) NOT NULL COMMENT "",
|
||||
`LO_LINENUMBER` tinyint(4) NOT NULL COMMENT "",
|
||||
`LO_CUSTKEY` int(11) NOT NULL COMMENT "",
|
||||
`LO_PARTKEY` int(11) NOT NULL COMMENT "",
|
||||
`LO_SUPPKEY` int(11) NOT NULL COMMENT "",
|
||||
`LO_ORDERPRIORITY` varchar(100) NOT NULL COMMENT "",
|
||||
`LO_SHIPPRIORITY` tinyint(4) NOT NULL COMMENT "",
|
||||
`LO_QUANTITY` tinyint(4) NOT NULL COMMENT "",
|
||||
`LO_EXTENDEDPRICE` int(11) NOT NULL COMMENT "",
|
||||
`LO_ORDTOTALPRICE` int(11) NOT NULL COMMENT "",
|
||||
`LO_DISCOUNT` tinyint(4) NOT NULL COMMENT "",
|
||||
`LO_REVENUE` int(11) NOT NULL COMMENT "",
|
||||
`LO_SUPPLYCOST` int(11) NOT NULL COMMENT "",
|
||||
`LO_TAX` tinyint(4) NOT NULL COMMENT "",
|
||||
`LO_COMMITDATE` date NOT NULL COMMENT "",
|
||||
`LO_SHIPMODE` varchar(100) NOT NULL COMMENT "",
|
||||
`C_NAME` varchar(100) NOT NULL COMMENT "",
|
||||
`C_ADDRESS` varchar(100) NOT NULL COMMENT "",
|
||||
`C_CITY` varchar(100) NOT NULL COMMENT "",
|
||||
`C_NATION` varchar(100) NOT NULL COMMENT "",
|
||||
`C_REGION` varchar(100) NOT NULL COMMENT "",
|
||||
`C_PHONE` varchar(100) NOT NULL COMMENT "",
|
||||
`C_MKTSEGMENT` varchar(100) NOT NULL COMMENT "",
|
||||
`S_NAME` varchar(100) NOT NULL COMMENT "",
|
||||
`S_ADDRESS` varchar(100) NOT NULL COMMENT "",
|
||||
`S_CITY` varchar(100) NOT NULL COMMENT "",
|
||||
`S_NATION` varchar(100) NOT NULL COMMENT "",
|
||||
`S_REGION` varchar(100) NOT NULL COMMENT "",
|
||||
`S_PHONE` varchar(100) NOT NULL COMMENT "",
|
||||
`P_NAME` varchar(100) NOT NULL COMMENT "",
|
||||
`P_MFGR` varchar(100) NOT NULL COMMENT "",
|
||||
`P_CATEGORY` varchar(100) NOT NULL COMMENT "",
|
||||
`P_BRAND` varchar(100) NOT NULL COMMENT "",
|
||||
`P_COLOR` varchar(100) NOT NULL COMMENT "",
|
||||
`P_TYPE` varchar(100) NOT NULL COMMENT "",
|
||||
`P_SIZE` tinyint(4) NOT NULL COMMENT "",
|
||||
`P_CONTAINER` varchar(100) NOT NULL COMMENT ""
|
||||
) ENGINE=OLAP
|
||||
UNIQUE KEY(`LO_ORDERDATE`, `LO_ORDERKEY`, `LO_LINENUMBER`)
|
||||
DISTRIBUTED BY HASH(`LO_ORDERKEY`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
@ -0,0 +1 @@
|
||||
truncate table lineorder_flat;
|
||||
@ -0,0 +1,16 @@
|
||||
CREATE TABLE IF NOT EXISTS `part` (
|
||||
`p_partkey` int(11) NOT NULL COMMENT "",
|
||||
`p_name` varchar(23) NOT NULL COMMENT "",
|
||||
`p_mfgr` varchar(7) NOT NULL COMMENT "",
|
||||
`p_category` varchar(8) NOT NULL COMMENT "",
|
||||
`p_brand` varchar(10) NOT NULL COMMENT "",
|
||||
`p_color` varchar(12) NOT NULL COMMENT "",
|
||||
`p_type` varchar(26) NOT NULL COMMENT "",
|
||||
`p_size` int(11) NOT NULL COMMENT "",
|
||||
`p_container` varchar(11) NOT NULL COMMENT ""
|
||||
)
|
||||
UNIQUE KEY (`p_partkey`)
|
||||
DISTRIBUTED BY HASH(`p_partkey`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
@ -0,0 +1 @@
|
||||
truncate table `part`;
|
||||
@ -0,0 +1,14 @@
|
||||
CREATE TABLE IF NOT EXISTS `supplier` (
|
||||
`s_suppkey` int(11) NOT NULL COMMENT "",
|
||||
`s_name` varchar(26) NOT NULL COMMENT "",
|
||||
`s_address` varchar(26) NOT NULL COMMENT "",
|
||||
`s_city` varchar(11) NOT NULL COMMENT "",
|
||||
`s_nation` varchar(16) NOT NULL COMMENT "",
|
||||
`s_region` varchar(13) NOT NULL COMMENT "",
|
||||
`s_phone` varchar(16) NOT NULL COMMENT ""
|
||||
)
|
||||
UNIQUE KEY (`s_suppkey`)
|
||||
DISTRIBUTED BY HASH(`s_suppkey`) BUCKETS 1
|
||||
PROPERTIES (
|
||||
"replication_num" = "1"
|
||||
);
|
||||
@ -0,0 +1 @@
|
||||
truncate table `supplier`;
|
||||
@ -0,0 +1,532 @@
|
||||
// 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.
|
||||
|
||||
// Most of the cases are copied from https://github.com/trinodb/trino/tree/master
|
||||
// /testing/trino-product-tests/src/main/resources/sql-tests/testcases
|
||||
// and modified by Doris.
|
||||
|
||||
// Note: To filter out tables from sql files, use the following one-liner comamnd
|
||||
// sed -nr 's/.*tables: (.*)$/\1/gp' /path/to/*.sql | sed -nr 's/,/\n/gp' | sort | uniq
|
||||
suite("mv_ssb_test") {
|
||||
|
||||
// ssb_sf1_p1 is writted to test unique key table merge correctly.
|
||||
// It creates unique key table and sets bucket num to 1 in order to make sure that
|
||||
// many rowsets will be created during loading and then the merge process will be triggered.
|
||||
|
||||
def tables = ["customer", "lineorder", "part", "date", "supplier"]
|
||||
def columns = ["""c_custkey,c_name,c_address,c_city,c_nation,c_region,c_phone,c_mktsegment,no_use""",
|
||||
"""lo_orderkey,lo_linenumber,lo_custkey,lo_partkey,lo_suppkey,lo_orderdate,lo_orderpriority,
|
||||
lo_shippriority,lo_quantity,lo_extendedprice,lo_ordtotalprice,lo_discount,
|
||||
lo_revenue,lo_supplycost,lo_tax,lo_commitdate,lo_shipmode,lo_dummy""",
|
||||
"""p_partkey,p_name,p_mfgr,p_category,p_brand,p_color,p_type,p_size,p_container,p_dummy""",
|
||||
"""d_datekey,d_date,d_dayofweek,d_month,d_year,d_yearmonthnum,d_yearmonth,
|
||||
d_daynuminweek,d_daynuminmonth,d_daynuminyear,d_monthnuminyear,d_weeknuminyear,
|
||||
d_sellingseason,d_lastdayinweekfl,d_lastdayinmonthfl,d_holidayfl,d_weekdayfl,d_dummy""",
|
||||
"""s_suppkey,s_name,s_address,s_city,s_nation,s_region,s_phone,s_dummy"""]
|
||||
|
||||
for (String table in tables) {
|
||||
sql new File("""${context.file.parent}/ddl/${table}_create.sql""").text
|
||||
sql new File("""${context.file.parent}/ddl/${table}_delete.sql""").text
|
||||
}
|
||||
def i = 0
|
||||
for (String tableName in tables) {
|
||||
streamLoad {
|
||||
// a default db 'regression_test' is specified in
|
||||
// ${DORIS_HOME}/conf/regression-conf.groovy
|
||||
table tableName
|
||||
|
||||
// default label is UUID:
|
||||
// set 'label' UUID.randomUUID().toString()
|
||||
|
||||
// default column_separator is specify in doris fe config, usually is '\t'.
|
||||
// this line change to ','
|
||||
set 'column_separator', '|'
|
||||
set 'compress_type', 'GZ'
|
||||
set 'columns', columns[i]
|
||||
|
||||
|
||||
// relate to ${DORIS_HOME}/regression-test/data/demo/streamload_input.csv.
|
||||
// also, you can stream load a http stream, e.g. http://xxx/some.csv
|
||||
file """${getS3Url()}/regression/ssb/sf1/${tableName}.tbl.gz"""
|
||||
|
||||
time 10000 // limit inflight 10s
|
||||
|
||||
// stream load action will check result, include Success status, and NumberTotalRows == NumberLoadedRows
|
||||
|
||||
// if declared a check callback, the default check condition will ignore.
|
||||
// So you must check all condition
|
||||
check { result, exception, startTime, endTime ->
|
||||
if (exception != null) {
|
||||
throw exception
|
||||
}
|
||||
log.info("Stream load result: ${result}".toString())
|
||||
def json = parseJson(result)
|
||||
assertEquals("success", json.Status.toLowerCase())
|
||||
assertEquals(json.NumberTotalRows, json.NumberLoadedRows)
|
||||
assertTrue(json.NumberLoadedRows > 0 && json.LoadBytes > 0)
|
||||
}
|
||||
}
|
||||
i++
|
||||
}
|
||||
sql """ sync """
|
||||
|
||||
String db = context.config.getDbNameByFile(context.file)
|
||||
sql "use ${db}"
|
||||
sql "SET enable_nereids_planner=true"
|
||||
sql "set runtime_filter_mode=OFF"
|
||||
sql "SET enable_fallback_to_original_planner=false"
|
||||
sql "SET enable_materialized_view_rewrite=true"
|
||||
sql "SET enable_nereids_timeout = false"
|
||||
|
||||
def check_rewrite = { mv_sql, query_sql, mv_name ->
|
||||
|
||||
sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}"""
|
||||
sql"""
|
||||
CREATE MATERIALIZED VIEW ${mv_name}
|
||||
BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL
|
||||
DISTRIBUTED BY RANDOM BUCKETS 2
|
||||
PROPERTIES ('replication_num' = '1')
|
||||
AS ${mv_sql}
|
||||
"""
|
||||
|
||||
def job_name = getJobName(db, mv_name);
|
||||
waitingMTMVTaskFinished(job_name)
|
||||
explain {
|
||||
sql("${query_sql}")
|
||||
contains("${mv_name}(${mv_name})")
|
||||
}
|
||||
}
|
||||
|
||||
def check_not_match = { mv_sql, query_sql, mv_name ->
|
||||
|
||||
sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}"""
|
||||
sql"""
|
||||
CREATE MATERIALIZED VIEW ${mv_name}
|
||||
BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL
|
||||
DISTRIBUTED BY RANDOM BUCKETS 2
|
||||
PROPERTIES ('replication_num' = '1')
|
||||
AS ${mv_sql}
|
||||
"""
|
||||
|
||||
def job_name = getJobName(db, mv_name);
|
||||
waitingMTMVTaskFinished(job_name)
|
||||
explain {
|
||||
sql("${query_sql}")
|
||||
notContains("${mv_name}(${mv_name})")
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
def mv1_1 = """
|
||||
SELECT SUM(lo_extendedprice*lo_discount) AS
|
||||
REVENUE
|
||||
FROM lineorder, date
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND d_year = 1993
|
||||
AND lo_discount BETWEEN 1 AND 3
|
||||
AND lo_quantity < 25;
|
||||
"""
|
||||
def query1_1 = """
|
||||
SELECT SUM(lo_extendedprice*lo_discount) AS
|
||||
REVENUE
|
||||
FROM lineorder, date
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND d_year = 1993
|
||||
AND lo_discount BETWEEN 1 AND 3
|
||||
AND lo_quantity < 25;
|
||||
"""
|
||||
order_qt_query1_1_before "${query1_1}"
|
||||
check_rewrite(mv1_1, query1_1, "mv1_1")
|
||||
order_qt_query1_1_after "${query1_1}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_1"""
|
||||
|
||||
|
||||
def mv1_2 = """
|
||||
SELECT SUM(lo_extendedprice*lo_discount) AS
|
||||
REVENUE
|
||||
FROM lineorder, date
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND d_yearmonth = 'Jan1994'
|
||||
AND lo_discount BETWEEN 4 AND 6
|
||||
AND lo_quantity BETWEEN 26 AND 35;
|
||||
"""
|
||||
def query1_2 = """
|
||||
SELECT SUM(lo_extendedprice*lo_discount) AS
|
||||
REVENUE
|
||||
FROM lineorder, date
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND d_yearmonth = 'Jan1994'
|
||||
AND lo_discount BETWEEN 4 AND 6
|
||||
AND lo_quantity BETWEEN 26 AND 35;
|
||||
"""
|
||||
order_qt_query1_2_before "${query1_2}"
|
||||
check_rewrite(mv1_2, query1_2, "mv1_2")
|
||||
order_qt_query1_2_after "${query1_2}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_2"""
|
||||
|
||||
def mv1_3 = """
|
||||
SELECT SUM(lo_extendedprice*lo_discount) AS
|
||||
REVENUE
|
||||
FROM lineorder, date
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND d_weeknuminyear= 6
|
||||
AND d_year = 1994
|
||||
AND lo_discount BETWEEN 5 AND 7
|
||||
AND lo_quantity BETWEEN 26 AND 35;
|
||||
"""
|
||||
def query1_3 = """
|
||||
SELECT SUM(lo_extendedprice*lo_discount) AS
|
||||
REVENUE
|
||||
FROM lineorder, date
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND d_weeknuminyear= 6
|
||||
AND d_year = 1994
|
||||
AND lo_discount BETWEEN 5 AND 7
|
||||
AND lo_quantity BETWEEN 26 AND 35;
|
||||
"""
|
||||
order_qt_query1_3before "${query1_3}"
|
||||
check_rewrite(mv1_3, query1_3, "mv1_3")
|
||||
order_qt_query1_3_after "${query1_3}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_3"""
|
||||
|
||||
|
||||
def mv2_1 = """
|
||||
SELECT SUM(lo_revenue), d_year, p_brand
|
||||
FROM lineorder, date, part, supplier
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND p_category = 'MFGR#12'
|
||||
AND s_region = 'AMERICA'
|
||||
GROUP BY d_year, p_brand
|
||||
ORDER BY d_year, p_brand;
|
||||
"""
|
||||
def query2_1 = """
|
||||
SELECT SUM(lo_revenue), d_year, p_brand
|
||||
FROM lineorder, date, part, supplier
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND p_category = 'MFGR#12'
|
||||
AND s_region = 'AMERICA'
|
||||
GROUP BY d_year, p_brand
|
||||
ORDER BY d_year, p_brand;
|
||||
"""
|
||||
order_qt_query2_1before "${query2_1}"
|
||||
check_rewrite(mv2_1, query2_1, "mv2_1")
|
||||
order_qt_query2_1_after "${query2_1}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_1"""
|
||||
|
||||
|
||||
def mv2_2 = """
|
||||
SELECT SUM(lo_revenue), d_year, p_brand
|
||||
FROM lineorder, date, part, supplier
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND p_brand BETWEEN 'MFGR#2221'
|
||||
AND 'MFGR#2228'
|
||||
AND s_region = 'ASIA'
|
||||
GROUP BY d_year, p_brand
|
||||
ORDER BY d_year, p_brand;
|
||||
"""
|
||||
def query2_2 = """
|
||||
SELECT SUM(lo_revenue), d_year, p_brand
|
||||
FROM lineorder, date, part, supplier
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND p_brand BETWEEN 'MFGR#2221'
|
||||
AND 'MFGR#2228'
|
||||
AND s_region = 'ASIA'
|
||||
GROUP BY d_year, p_brand
|
||||
ORDER BY d_year, p_brand;
|
||||
"""
|
||||
order_qt_query2_2before "${query2_2}"
|
||||
check_rewrite(mv2_2, query2_2, "mv2_2")
|
||||
order_qt_query2_2_after "${query2_2}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_2"""
|
||||
|
||||
def mv2_3 = """
|
||||
SELECT SUM(lo_revenue), d_year, p_brand
|
||||
FROM lineorder, date, part, supplier
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND p_brand = 'MFGR#2239'
|
||||
AND s_region = 'EUROPE'
|
||||
GROUP BY d_year, p_brand
|
||||
ORDER BY d_year, p_brand;
|
||||
"""
|
||||
def query2_3 = """
|
||||
SELECT SUM(lo_revenue), d_year, p_brand
|
||||
FROM lineorder, date, part, supplier
|
||||
WHERE lo_orderdate = d_datekey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND p_brand = 'MFGR#2239'
|
||||
AND s_region = 'EUROPE'
|
||||
GROUP BY d_year, p_brand
|
||||
ORDER BY d_year, p_brand;
|
||||
"""
|
||||
order_qt_query2_3before "${query2_3}"
|
||||
check_rewrite(mv2_3, query2_3, "mv2_3")
|
||||
order_qt_query2_3_after "${query2_3}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_3"""
|
||||
|
||||
def mv3_1 = """
|
||||
SELECT c_nation, s_nation, d_year,
|
||||
SUM(lo_revenue) AS REVENUE
|
||||
FROM customer, lineorder, supplier, date
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND c_region = 'ASIA'
|
||||
AND s_region = 'ASIA'
|
||||
AND d_year >= 1992 AND d_year <= 1997
|
||||
GROUP BY c_nation, s_nation, d_year
|
||||
ORDER BY d_year ASC, REVENUE DESC;
|
||||
"""
|
||||
def query3_1 = """
|
||||
SELECT c_nation, s_nation, d_year,
|
||||
SUM(lo_revenue) AS REVENUE
|
||||
FROM customer, lineorder, supplier, date
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND c_region = 'ASIA'
|
||||
AND s_region = 'ASIA'
|
||||
AND d_year >= 1992 AND d_year <= 1997
|
||||
GROUP BY c_nation, s_nation, d_year
|
||||
ORDER BY d_year ASC, REVENUE DESC;
|
||||
"""
|
||||
order_qt_query3_1before "${query3_1}"
|
||||
check_rewrite(mv3_1, query3_1, "mv3_1")
|
||||
order_qt_query3_1_after "${query3_1}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_1"""
|
||||
|
||||
|
||||
def mv3_2 = """
|
||||
SELECT c_city, s_city, d_year, sum(lo_revenue)
|
||||
AS REVENUE
|
||||
FROM customer, lineorder, supplier, date
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND c_nation = 'UNITED STATES'
|
||||
AND s_nation = 'UNITED STATES'
|
||||
AND d_year >= 1992 AND d_year <= 1997
|
||||
GROUP BY c_city, s_city, d_year
|
||||
ORDER BY d_year ASC, REVENUE DESC;
|
||||
"""
|
||||
def query3_2 = """
|
||||
SELECT c_city, s_city, d_year, sum(lo_revenue)
|
||||
AS REVENUE
|
||||
FROM customer, lineorder, supplier, date
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND c_nation = 'UNITED STATES'
|
||||
AND s_nation = 'UNITED STATES'
|
||||
AND d_year >= 1992 AND d_year <= 1997
|
||||
GROUP BY c_city, s_city, d_year
|
||||
ORDER BY d_year ASC, REVENUE DESC;
|
||||
"""
|
||||
order_qt_query3_2before "${query3_2}"
|
||||
check_rewrite(mv3_2, query3_2, "mv3_2")
|
||||
order_qt_query3_2_after "${query3_2}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_2"""
|
||||
|
||||
|
||||
def mv3_3 = """
|
||||
SELECT c_city, s_city, d_year, SUM(lo_revenue)
|
||||
AS REVENUE
|
||||
FROM customer, lineorder, supplier, date
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND (c_city='UNITED KI1'
|
||||
OR c_city='UNITED KI5')
|
||||
AND (s_city='UNITED KI1'
|
||||
OR s_city='UNITED KI5')
|
||||
AND d_year >= 1992 AND d_year <= 1997
|
||||
GROUP BY c_city, s_city, d_year
|
||||
ORDER BY d_year ASC, REVENUE DESC;
|
||||
"""
|
||||
def query3_3 = """
|
||||
SELECT c_city, s_city, d_year, SUM(lo_revenue)
|
||||
AS REVENUE
|
||||
FROM customer, lineorder, supplier, date
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND (c_city='UNITED KI1'
|
||||
OR c_city='UNITED KI5')
|
||||
AND (s_city='UNITED KI1'
|
||||
OR s_city='UNITED KI5')
|
||||
AND d_year >= 1992 AND d_year <= 1997
|
||||
GROUP BY c_city, s_city, d_year
|
||||
ORDER BY d_year ASC, REVENUE DESC;
|
||||
"""
|
||||
order_qt_query3_3before "${query3_3}"
|
||||
check_rewrite(mv3_3, query3_3, "mv3_3")
|
||||
order_qt_query3_3_after "${query3_3}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_3"""
|
||||
|
||||
|
||||
def mv3_4 = """
|
||||
SELECT c_city, s_city, d_year, SUM(lo_revenue)
|
||||
AS REVENUE
|
||||
FROM customer, lineorder, supplier, date
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND (c_city='UNITED KI1'
|
||||
OR c_city='UNITED KI5')
|
||||
AND (s_city='UNITED KI1'
|
||||
OR s_city='UNITED KI5')
|
||||
AND d_yearmonth = 'Dec1997'
|
||||
GROUP BY c_city, s_city, d_year
|
||||
ORDER BY d_year ASC, REVENUE DESC;
|
||||
"""
|
||||
def query3_4 = """
|
||||
SELECT c_city, s_city, d_year, SUM(lo_revenue)
|
||||
AS REVENUE
|
||||
FROM customer, lineorder, supplier, date
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND (c_city='UNITED KI1'
|
||||
OR c_city='UNITED KI5')
|
||||
AND (s_city='UNITED KI1'
|
||||
OR s_city='UNITED KI5')
|
||||
AND d_yearmonth = 'Dec1997'
|
||||
GROUP BY c_city, s_city, d_year
|
||||
ORDER BY d_year ASC, REVENUE DESC;
|
||||
"""
|
||||
order_qt_query3_4before "${query3_4}"
|
||||
check_rewrite(mv3_4, query3_4, "mv3_4")
|
||||
order_qt_query3_4_after "${query3_4}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_4"""
|
||||
|
||||
|
||||
def mv4_1 = """
|
||||
SELECT d_year, c_nation,
|
||||
SUM(lo_revenue - lo_supplycost) AS PROFIT
|
||||
FROM date, customer, supplier, part, lineorder
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND c_region = 'AMERICA'
|
||||
AND s_region = 'AMERICA'
|
||||
AND (p_mfgr = 'MFGR#1'
|
||||
OR p_mfgr = 'MFGR#2')
|
||||
GROUP BY d_year, c_nation
|
||||
ORDER BY d_year, c_nation;
|
||||
"""
|
||||
def query4_1 = """
|
||||
SELECT d_year, c_nation,
|
||||
SUM(lo_revenue - lo_supplycost) AS PROFIT
|
||||
FROM date, customer, supplier, part, lineorder
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND c_region = 'AMERICA'
|
||||
AND s_region = 'AMERICA'
|
||||
AND (p_mfgr = 'MFGR#1'
|
||||
OR p_mfgr = 'MFGR#2')
|
||||
GROUP BY d_year, c_nation
|
||||
ORDER BY d_year, c_nation;
|
||||
"""
|
||||
order_qt_query4_1before "${query4_1}"
|
||||
check_rewrite(mv4_1, query4_1, "mv4_1")
|
||||
order_qt_query4_1_after "${query4_1}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_1"""
|
||||
|
||||
def mv4_2 = """
|
||||
SELECT d_year, s_nation, p_category,
|
||||
SUM(lo_revenue - lo_supplycost) AS PROFIT
|
||||
FROM date, customer, supplier, part, lineorder
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND c_region = 'AMERICA'
|
||||
AND s_region = 'AMERICA'
|
||||
AND (d_year = 1997 OR d_year = 1998)
|
||||
AND (p_mfgr = 'MFGR#1'
|
||||
OR p_mfgr = 'MFGR#2')
|
||||
GROUP BY d_year, s_nation, p_category
|
||||
ORDER BY d_year, s_nation, p_category;
|
||||
"""
|
||||
def query4_2 = """
|
||||
SELECT d_year, s_nation, p_category,
|
||||
SUM(lo_revenue - lo_supplycost) AS PROFIT
|
||||
FROM date, customer, supplier, part, lineorder
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND c_region = 'AMERICA'
|
||||
AND s_region = 'AMERICA'
|
||||
AND (d_year = 1997 OR d_year = 1998)
|
||||
AND (p_mfgr = 'MFGR#1'
|
||||
OR p_mfgr = 'MFGR#2')
|
||||
GROUP BY d_year, s_nation, p_category
|
||||
ORDER BY d_year, s_nation, p_category;
|
||||
"""
|
||||
order_qt_query4_2before "${query4_2}"
|
||||
check_rewrite(mv4_2, query4_2, "mv4_2")
|
||||
order_qt_query4_2_after "${query4_2}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_2"""
|
||||
|
||||
|
||||
def mv4_3 = """
|
||||
SELECT d_year, s_city, p_brand,
|
||||
SUM(lo_revenue - lo_supplycost) AS PROFIT
|
||||
FROM date, customer, supplier, part, lineorder
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND s_nation = 'UNITED STATES'
|
||||
AND (d_year = 1997 OR d_year = 1998)
|
||||
AND p_category = 'MFGR#14'
|
||||
GROUP BY d_year, s_city, p_brand
|
||||
ORDER BY d_year, s_city, p_brand;
|
||||
"""
|
||||
def query4_3 = """
|
||||
SELECT d_year, s_city, p_brand,
|
||||
SUM(lo_revenue - lo_supplycost) AS PROFIT
|
||||
FROM date, customer, supplier, part, lineorder
|
||||
WHERE lo_custkey = c_custkey
|
||||
AND lo_suppkey = s_suppkey
|
||||
AND lo_partkey = p_partkey
|
||||
AND lo_orderdate = d_datekey
|
||||
AND s_nation = 'UNITED STATES'
|
||||
AND (d_year = 1997 OR d_year = 1998)
|
||||
AND p_category = 'MFGR#14'
|
||||
GROUP BY d_year, s_city, p_brand
|
||||
ORDER BY d_year, s_city, p_brand;
|
||||
"""
|
||||
order_qt_query4_3before "${query4_3}"
|
||||
check_rewrite(mv4_3, query4_3, "mv4_3")
|
||||
order_qt_query4_3_after "${query4_3}"
|
||||
sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_3"""
|
||||
}
|
||||
Reference in New Issue
Block a user