[feat](Nereids) support outer join and aggregate bitmap rewrite by mv (#28596)

- Support left outer join rewrite by materialized view
- Support bitmap_union roll up to imp count(distinct)
- Support partition materialized view rewrite
This commit is contained in:
seawinde
2023-12-20 10:23:30 +08:00
committed by GitHub
parent 17268346d3
commit 4c0080e237
42 changed files with 3680 additions and 454 deletions

View File

@ -35,6 +35,7 @@ import org.apache.doris.mtmv.MTMVRefreshInfo;
import org.apache.doris.mtmv.MTMVRelation;
import org.apache.doris.mtmv.MTMVStatus;
import org.apache.doris.persist.gson.GsonUtils;
import org.apache.doris.qe.ConnectContext;
import com.google.common.collect.Sets;
import com.google.gson.annotations.SerializedName;
@ -128,10 +129,6 @@ public class MTMV extends OlapTable {
return relation;
}
public MTMVCache getCache() {
return cache;
}
public void setCache(MTMVCache cache) {
this.cache = cache;
}
@ -202,12 +199,13 @@ public class MTMV extends OlapTable {
return Sets.newHashSet(split);
}
public MTMVCache getOrGenerateCache() throws AnalysisException {
// this should use the same connectContext with query, to use the same session variable
public MTMVCache getOrGenerateCache(ConnectContext parent) throws AnalysisException {
if (cache == null) {
writeMvLock();
try {
if (cache == null) {
this.cache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this));
this.cache = MTMVCache.from(this, parent);
}
} finally {
writeMvUnlock();

View File

@ -74,7 +74,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 = mvRewrittenPlan.getExpressions().stream()
List<NamedExpression> mvOutputExpressions = mvPlan.getExpressions().stream()
.map(NamedExpression.class::cast)
.collect(Collectors.toList());
return new MTMVCache(mvPlan, mvOutputExpressions);

View File

@ -30,6 +30,7 @@ import org.apache.doris.nereids.trees.plans.commands.info.RefreshMTMVInfo;
import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo;
import org.apache.doris.persist.AlterMTMV;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import org.apache.commons.collections.CollectionUtils;
@ -49,7 +50,7 @@ public class MTMVRelationManager implements MTMVHookService {
private Map<BaseTableInfo, Set<BaseTableInfo>> tableMTMVs = Maps.newConcurrentMap();
public Set<BaseTableInfo> getMtmvsByBaseTable(BaseTableInfo table) {
return tableMTMVs.get(table);
return tableMTMVs.getOrDefault(table, ImmutableSet.of());
}
public Set<MTMV> getAvailableMTMVs(List<BaseTableInfo> tableInfos) {

View File

@ -253,7 +253,7 @@ public class MTMVUtil {
List<Partition> res = Lists.newArrayList();
Collection<Partition> allPartitions = mtmv.getPartitions();
// check session variable if enable rewrite
if (!ctx.getSessionVariable().isEnableMvRewrite()) {
if (!ctx.getSessionVariable().isEnableMaterializedViewRewrite()) {
return res;
}
MTMVRelation mtmvRelation = mtmv.getRelation();
@ -438,7 +438,7 @@ public class MTMVUtil {
* @param relatedTable
* @return mv.partitionId ==> relatedTable.partitionId
*/
private static Map<Long, Set<Long>> getMvToBasePartitions(MTMV mtmv, OlapTable relatedTable)
public static Map<Long, Set<Long>> getMvToBasePartitions(MTMV mtmv, OlapTable relatedTable)
throws AnalysisException {
HashMap<Long, Set<Long>> res = Maps.newHashMap();
Map<Long, PartitionItem> relatedTableItems = relatedTable.getPartitionInfo().getIdToItem(false);

View File

@ -318,7 +318,9 @@ public class CascadesContext implements ScheduleContext {
}
public List<MaterializationContext> getMaterializationContexts() {
return materializationContexts;
return materializationContexts.stream()
.filter(MaterializationContext::isAvailable)
.collect(Collectors.toList());
}
public void addMaterializationContext(MaterializationContext materializationContext) {

View File

@ -33,7 +33,6 @@ import java.util.Objects;
* @see PlaceholderCollector
*/
public class PlaceholderExpression extends Expression implements AlwaysNotNullable {
private final Class<? extends Expression> delegateClazz;
/**
* 1 based

View File

@ -40,7 +40,11 @@ import org.apache.doris.nereids.rules.exploration.join.PushDownProjectThroughSem
import org.apache.doris.nereids.rules.exploration.join.SemiJoinSemiJoinTranspose;
import org.apache.doris.nereids.rules.exploration.join.SemiJoinSemiJoinTransposeProject;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewAggregateRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewFilterJoinRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewFilterProjectJoinRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewOnlyJoinRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectAggregateRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectFilterJoinRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectJoinRule;
import org.apache.doris.nereids.rules.implementation.AggregateStrategies;
import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows;
@ -222,7 +226,11 @@ public class RuleSet {
.build();
public static final List<Rule> MATERIALIZED_VIEW_RULES = planRuleFactories()
.add(MaterializedViewOnlyJoinRule.INSTANCE)
.add(MaterializedViewProjectJoinRule.INSTANCE)
.add(MaterializedViewFilterJoinRule.INSTANCE)
.add(MaterializedViewFilterProjectJoinRule.INSTANCE)
.add(MaterializedViewProjectFilterJoinRule.INSTANCE)
.add(MaterializedViewAggregateRule.INSTANCE)
.add(MaterializedViewProjectAggregateRule.INSTANCE)
.build();

View File

@ -25,12 +25,16 @@ import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode;
import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PlanSplitContext;
import org.apache.doris.nereids.rules.exploration.mv.mapping.ExpressionMapping;
import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping;
import org.apache.doris.nereids.trees.expressions.Any;
import org.apache.doris.nereids.trees.expressions.ExprId;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.functions.Function;
import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnion;
import org.apache.doris.nereids.trees.expressions.functions.agg.CouldRollUp;
import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
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;
@ -39,8 +43,11 @@ import org.apache.doris.nereids.util.ExpressionUtils;
import com.google.common.collect.HashMultimap;
import com.google.common.collect.Multimap;
import com.google.common.collect.Sets;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@ -53,6 +60,17 @@ import java.util.stream.Collectors;
*/
public abstract class AbstractMaterializedViewAggregateRule extends AbstractMaterializedViewRule {
protected static final Map<Expression, Expression>
AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP = new HashMap<>();
protected final String currentClassName = this.getClass().getSimpleName();
private final Logger logger = LogManager.getLogger(this.getClass());
static {
AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.put(new Count(true, Any.INSTANCE),
new BitmapUnion(Any.INSTANCE));
}
@Override
protected Plan rewriteQueryByView(MatchMode matchMode,
StructInfo queryStructInfo,
@ -63,10 +81,12 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
// get view and query aggregate and top plan correspondingly
Pair<Plan, LogicalAggregate<Plan>> viewTopPlanAndAggPair = splitToTopPlanAndAggregate(viewStructInfo);
if (viewTopPlanAndAggPair == null) {
logger.warn(currentClassName + " split to view to top plan and agg fail so return null");
return null;
}
Pair<Plan, LogicalAggregate<Plan>> queryTopPlanAndAggPair = splitToTopPlanAndAggregate(queryStructInfo);
if (queryTopPlanAndAggPair == null) {
logger.warn(currentClassName + " split to query to top plan and agg fail so return null");
return null;
}
// Firstly, handle query group by expression rewrite
@ -88,13 +108,14 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
needRollUp = !queryGroupShuttledExpression.equals(viewGroupShuttledExpression);
}
if (!needRollUp) {
List<Expression> rewrittenQueryGroupExpr = rewriteExpression(queryTopPlan.getOutput(),
List<Expression> rewrittenQueryGroupExpr = rewriteExpression(queryTopPlan.getExpressions(),
queryTopPlan,
materializationContext.getMvExprToMvScanExprMapping(),
queryToViewSlotMapping,
true);
if (rewrittenQueryGroupExpr == null) {
if (rewrittenQueryGroupExpr.isEmpty()) {
// can not rewrite, bail out.
logger.debug(currentClassName + " can not rewrite expression when not need roll up");
return null;
}
return new LogicalProject<>(
@ -109,12 +130,14 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
viewExpr -> viewExpr.anyMatch(expr -> expr instanceof AggregateFunction
&& ((AggregateFunction) expr).isDistinct()))) {
// if mv aggregate function contains distinct, can not roll up, bail out.
logger.debug(currentClassName + " view contains distinct function so can not roll up");
return null;
}
// split the query top plan expressions to group expressions and functions, if can not, bail out.
Pair<Set<? extends Expression>, Set<? extends Expression>> queryGroupAndFunctionPair
= topPlanSplitToGroupAndFunction(queryTopPlanAndAggPair);
if (queryGroupAndFunctionPair == null) {
logger.warn(currentClassName + " query top plan split to group by and function fail so return null");
return null;
}
// Secondly, try to roll up the agg functions
@ -132,23 +155,19 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
for (Expression topExpression : queryTopPlan.getExpressions()) {
// is agg function, try to roll up and rewrite
if (queryTopPlanFunctionSet.contains(topExpression)) {
Expression needRollupShuttledExpr = ExpressionUtils.shuttleExpressionWithLineage(
Expression queryFunctionShuttled = ExpressionUtils.shuttleExpressionWithLineage(
topExpression,
queryTopPlan);
if (!mvExprToMvScanExprQueryBased.containsKey(needRollupShuttledExpr)) {
// function can not rewrite by view
return null;
}
// try to roll up
AggregateFunction needRollupAggFunction = (AggregateFunction) topExpression.firstMatch(
AggregateFunction queryFunction = (AggregateFunction) topExpression.firstMatch(
expr -> expr instanceof AggregateFunction);
AggregateFunction rollupAggregateFunction = rollup(needRollupAggFunction,
mvExprToMvScanExprQueryBased.get(needRollupShuttledExpr));
Function rollupAggregateFunction = rollup(queryFunction, queryFunctionShuttled,
mvExprToMvScanExprQueryBased);
if (rollupAggregateFunction == null) {
return null;
}
// key is query need roll up expr, value is mv scan based roll up expr
needRollupExprMap.put(needRollupShuttledExpr, rollupAggregateFunction);
needRollupExprMap.put(queryFunctionShuttled, rollupAggregateFunction);
// rewrite query function expression by mv expression
Expression rewrittenFunctionExpression = rewriteExpression(topExpression,
queryTopPlan,
@ -156,6 +175,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
queryToViewSlotMapping,
false);
if (rewrittenFunctionExpression == null) {
logger.debug(currentClassName + " roll up expression can not rewrite by view so return null");
return null;
}
finalAggregateExpressions.add((NamedExpression) rewrittenFunctionExpression);
@ -165,6 +185,8 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
ExpressionUtils.shuttleExpressionWithLineage(topExpression, queryTopPlan);
if (!mvExprToMvScanExprQueryBased.containsKey(queryGroupShuttledExpr)) {
// group expr can not rewrite by view
logger.debug(currentClassName
+ " view group expressions can not contains the query group by expression so return null");
return null;
}
groupRewrittenExprMap.put(queryGroupShuttledExpr,
@ -177,6 +199,8 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
queryToViewSlotMapping,
true);
if (rewrittenGroupExpression == null) {
logger.debug(currentClassName
+ " query top expression can not be rewritten by view so return null");
return null;
}
finalAggregateExpressions.add((NamedExpression) rewrittenGroupExpression);
@ -226,17 +250,33 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
}
// only support sum roll up, support other agg functions later.
private AggregateFunction rollup(AggregateFunction originFunction,
Expression mappedExpression) {
Class<? extends AggregateFunction> rollupAggregateFunction = originFunction.getRollup();
if (rollupAggregateFunction == null) {
private Function rollup(AggregateFunction queryFunction,
Expression queryFunctionShuttled,
Map<Expression, Expression> mvExprToMvScanExprQueryBased) {
if (!(queryFunction instanceof CouldRollUp)) {
return null;
}
if (Sum.class.isAssignableFrom(rollupAggregateFunction)) {
return new Sum(originFunction.isDistinct(), mappedExpression);
Expression rollupParam = null;
if (mvExprToMvScanExprQueryBased.containsKey(queryFunctionShuttled)) {
// function can rewrite by view
rollupParam = mvExprToMvScanExprQueryBased.get(queryFunctionShuttled);
} else {
// function can not rewrite by view, try to use complex roll up param
// eg: query is count(distinct param), mv sql is bitmap_union(to_bitmap(param))
for (Expression mvExprShuttled : mvExprToMvScanExprQueryBased.keySet()) {
if (!(mvExprShuttled instanceof Function)) {
continue;
}
if (isAggregateFunctionEquivalent(queryFunction, (Function) mvExprShuttled)) {
rollupParam = mvExprToMvScanExprQueryBased.get(mvExprShuttled);
}
}
}
// can rollup return null
return null;
if (rollupParam == null) {
return null;
}
// do roll up
return ((CouldRollUp) queryFunction).constructRollUp(rollupParam);
}
private Pair<Set<? extends Expression>, Set<? extends Expression>> topPlanSplitToGroupAndFunction(
@ -306,4 +346,23 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
}
return true;
}
private boolean isAggregateFunctionEquivalent(Function queryFunction, Function viewFunction) {
if (queryFunction.equals(viewFunction)) {
return true;
}
// get query equivalent function
Expression equivalentFunction = null;
for (Map.Entry<Expression, Expression> entry : AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.entrySet()) {
if (entry.getKey().equals(queryFunction)) {
equivalentFunction = entry.getValue();
}
}
// check is have equivalent function or not
if (equivalentFunction == null) {
return false;
}
// current compare
return equivalentFunction.equals(viewFunction);
}
}

View File

@ -27,6 +27,9 @@ 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;
@ -35,6 +38,10 @@ import java.util.stream.Collectors;
* This is responsible for common join rewriting
*/
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,
@ -53,6 +60,7 @@ public abstract class AbstractMaterializedViewJoinRule extends AbstractMateriali
// Can not rewrite, bail out
if (expressionsRewritten.isEmpty()
|| expressionsRewritten.stream().anyMatch(expr -> !(expr instanceof NamedExpression))) {
logger.warn(currentClassName + " expression to rewrite is not named expr so return null");
return null;
}
// record the group id in materializationContext, and when rewrite again in

View File

@ -17,8 +17,21 @@
package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.catalog.MTMV;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.Partition;
import org.apache.doris.catalog.PartitionInfo;
import org.apache.doris.catalog.PartitionItem;
import org.apache.doris.catalog.PartitionType;
import org.apache.doris.catalog.TableIf;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.mtmv.BaseTableInfo;
import org.apache.doris.mtmv.MTMVCache;
import org.apache.doris.mtmv.MTMVPartitionInfo;
import org.apache.doris.mtmv.MTMVUtil;
import org.apache.doris.nereids.CascadesContext;
import org.apache.doris.nereids.jobs.executor.Rewriter;
import org.apache.doris.nereids.rules.exploration.ExplorationRuleFactory;
import org.apache.doris.nereids.rules.exploration.mv.Predicates.SplitPredicate;
import org.apache.doris.nereids.rules.exploration.mv.mapping.EquivalenceClassSetMapping;
import org.apache.doris.nereids.rules.exploration.mv.mapping.ExpressionMapping;
@ -36,26 +49,32 @@ import org.apache.doris.nereids.trees.plans.JoinType;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation;
import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
import org.apache.doris.nereids.util.ExpressionUtils;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Sets;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
/**
* The abstract class for all materialized view rules
*/
public abstract class AbstractMaterializedViewRule {
public abstract class AbstractMaterializedViewRule implements ExplorationRuleFactory {
public static final HashSet<JoinType> SUPPORTED_JOIN_TYPE_SET =
Sets.newHashSet(JoinType.INNER_JOIN, JoinType.LEFT_OUTER_JOIN);
protected final String currentClassName = this.getClass().getSimpleName();
private final Logger logger = LogManager.getLogger(this.getClass());
/**
* The abstract template method for query rewrite, it contains the main logic and different query
@ -65,6 +84,7 @@ public abstract class AbstractMaterializedViewRule {
List<MaterializationContext> materializationContexts = cascadesContext.getMaterializationContexts();
List<Plan> rewriteResults = new ArrayList<>();
if (materializationContexts.isEmpty()) {
logger.info(currentClassName + " materializationContexts is empty so return");
return rewriteResults;
}
@ -72,6 +92,7 @@ public abstract class AbstractMaterializedViewRule {
// TODO Just Check query queryPlan firstly, support multi later.
StructInfo queryStructInfo = queryStructInfos.get(0);
if (!checkPattern(queryStructInfo)) {
logger.info(currentClassName + " queryStructInfo is not valid so return");
return rewriteResults;
}
@ -80,40 +101,51 @@ public abstract class AbstractMaterializedViewRule {
if (queryPlan.getGroupExpression().isPresent()
&& materializationContext.alreadyRewrite(
queryPlan.getGroupExpression().get().getOwnerGroup().getGroupId())) {
logger.info(currentClassName + " this group is already rewritten so skip");
continue;
}
Plan mvPlan = materializationContext.getMtmv().getCache().getLogicalPlan();
List<StructInfo> viewStructInfos = extractStructInfo(mvPlan, cascadesContext);
MTMV mtmv = materializationContext.getMTMV();
MTMVCache mtmvCache = getCacheFromMTMV(mtmv, cascadesContext);
if (mtmvCache == null) {
logger.info(currentClassName + " mv cache is null so return");
return rewriteResults;
}
List<StructInfo> viewStructInfos = extractStructInfo(mtmvCache.getLogicalPlan(), cascadesContext);
if (viewStructInfos.size() > 1) {
// view struct info should only have one
logger.info(currentClassName + " the num of view struct info is more then one so return");
return rewriteResults;
}
StructInfo viewStructInfo = viewStructInfos.get(0);
if (!checkPattern(viewStructInfo)) {
logger.info(currentClassName + " viewStructInfo is not valid so return");
continue;
}
MatchMode matchMode = decideMatchMode(queryStructInfo.getRelations(), viewStructInfo.getRelations());
if (MatchMode.COMPLETE != matchMode) {
logger.info(currentClassName + " match mode is not complete so return");
continue;
}
List<RelationMapping> queryToViewTableMappings =
RelationMapping.generate(queryStructInfo.getRelations(), viewStructInfo.getRelations());
// if any relation in query and view can not map, bail out.
if (queryToViewTableMappings == null) {
logger.info(currentClassName + " query to view table mapping null so return");
return rewriteResults;
}
for (RelationMapping queryToViewTableMapping : queryToViewTableMappings) {
SlotMapping queryToViewSlotMapping = SlotMapping.generate(queryToViewTableMapping);
if (queryToViewSlotMapping == null) {
logger.info(currentClassName + " query to view slot mapping null so continue");
continue;
}
LogicalCompatibilityContext compatibilityContext =
LogicalCompatibilityContext.from(queryToViewTableMapping, queryToViewSlotMapping,
queryStructInfo, viewStructInfo);
// todo outer join compatibility check
List<Expression> pulledUpExpressions = StructInfo.isGraphLogicalEquals(queryStructInfo, viewStructInfo,
compatibilityContext);
if (pulledUpExpressions == null) {
logger.info(currentClassName + " graph logical is not equals so continue");
continue;
}
// set pulled up expression to queryStructInfo predicates and update related predicates
@ -124,6 +156,7 @@ public abstract class AbstractMaterializedViewRule {
queryToViewSlotMapping);
// Can not compensate, bail out
if (compensatePredicates.isEmpty()) {
logger.info(currentClassName + " predicate compensate fail so continue");
continue;
}
Plan rewritedPlan;
@ -139,6 +172,7 @@ public abstract class AbstractMaterializedViewRule {
queryToViewSlotMapping,
true);
if (rewriteCompensatePredicates.isEmpty()) {
logger.info(currentClassName + " compensate predicate rewrite by view fail so continue");
continue;
}
rewritedPlan = new LogicalFilter<>(Sets.newHashSet(rewriteCompensatePredicates), mvScan);
@ -151,14 +185,108 @@ public abstract class AbstractMaterializedViewRule {
rewritedPlan,
materializationContext);
if (rewritedPlan == null) {
logger.info(currentClassName + " rewrite query by view fail so continue");
continue;
}
if (!checkPartitionIsValid(queryStructInfo, materializationContext, cascadesContext)) {
logger.info(currentClassName + " check partition validation fail so continue");
continue;
}
// run rbo job on mv rewritten plan
CascadesContext rewrittenPlanContext =
CascadesContext.initContext(cascadesContext.getStatementContext(), rewritedPlan,
cascadesContext.getCurrentJobContext().getRequiredProperties());
Rewriter.getWholeTreeRewriter(cascadesContext).execute();
rewritedPlan = rewrittenPlanContext.getRewritePlan();
logger.info(currentClassName + "rewrite by materialized view success");
rewriteResults.add(rewritedPlan);
}
}
return rewriteResults;
}
/**
* Partition will be pruned in query then add the record the partitions to select partitions on
* catalog relation.
* Maybe only just some partitions is valid in materialized view, so we should check if the mv can
* offer the partitions which query used or not.
*/
protected boolean checkPartitionIsValid(
StructInfo queryInfo,
MaterializationContext materializationContext,
CascadesContext cascadesContext) {
// check partition is valid or not
MTMV mtmv = materializationContext.getMTMV();
PartitionInfo mvPartitionInfo = mtmv.getPartitionInfo();
if (PartitionType.UNPARTITIONED.equals(mvPartitionInfo.getType())) {
// if not partition, if rewrite success, it means mv is available
return true;
}
// check mv related table partition is valid or not
MTMVPartitionInfo mvCustomPartitionInfo = mtmv.getMvPartitionInfo();
BaseTableInfo relatedPartitionTable = mvCustomPartitionInfo.getRelatedTable();
if (relatedPartitionTable == null) {
return true;
}
Optional<LogicalOlapScan> relatedTableRelation = queryInfo.getRelations().stream()
.filter(LogicalOlapScan.class::isInstance)
.filter(relation -> relatedPartitionTable.equals(new BaseTableInfo(relation.getTable())))
.map(LogicalOlapScan.class::cast)
.findFirst();
if (!relatedTableRelation.isPresent()) {
logger.warn("mv is partition update, but related table relation is null");
return false;
}
OlapTable relatedTable = relatedTableRelation.get().getTable();
Map<Long, Set<Long>> mvToBasePartitionMap;
try {
mvToBasePartitionMap = MTMVUtil.getMvToBasePartitions(mtmv, relatedTable);
} catch (AnalysisException e) {
logger.warn("mvRewriteSuccess getMvToBasePartitions fail", e);
return false;
}
// get mv valid partitions
Collection<Partition> mvDataValidPartitions = MTMVUtil.getMTMVCanRewritePartitions(mtmv,
cascadesContext.getConnectContext());
Map<Long, PartitionItem> allPartitions = mvPartitionInfo.getAllPartitions();
if (!allPartitions.isEmpty() && mvDataValidPartitions.isEmpty()) {
// do not have valid partition
return false;
}
// get mv related table valid partitions
Set<Long> relatedTalbeValidSet = mvDataValidPartitions.stream()
.map(partition -> {
Set<Long> relatedBaseTablePartitions = mvToBasePartitionMap.get(partition.getId());
if (relatedBaseTablePartitions == null || relatedBaseTablePartitions.isEmpty()) {
return ImmutableList.of();
} else {
return relatedBaseTablePartitions;
}
})
.flatMap(Collection::stream)
.map(Long.class::cast)
.collect(Collectors.toSet());
// get query selected partitions to make the partitions is valid or not
Set<Long> relatedTableSelectedPartitionToCheck =
new HashSet<>(relatedTableRelation.get().getSelectedPartitionIds());
if (relatedTableSelectedPartitionToCheck.isEmpty()) {
relatedTableSelectedPartitionToCheck.addAll(relatedTable.getPartitionIds());
}
return !relatedTalbeValidSet.isEmpty()
&& relatedTalbeValidSet.containsAll(relatedTableSelectedPartitionToCheck);
}
private MTMVCache getCacheFromMTMV(MTMV mtmv, CascadesContext cascadesContext) {
MTMVCache cache;
try {
cache = mtmv.getOrGenerateCache(cascadesContext.getConnectContext());
} catch (AnalysisException analysisException) {
logger.warn(this.getClass().getSimpleName() + " get mtmv cache analysisException", analysisException);
return null;
}
return cache;
}
/**
* Rewrite query by view, for aggregate or join rewriting should be different inherit class implementation
*/
@ -268,6 +396,7 @@ public abstract class AbstractMaterializedViewRule {
.toSlotReferenceMap();
EquivalenceClass viewEquivalenceClassQueryBased = viewEquivalenceClass.permute(viewToQuerySlotMapping);
if (viewEquivalenceClassQueryBased == null) {
logger.info(currentClassName + " permute view equivalence class by query fail so return empty");
return SplitPredicate.empty();
}
final List<Expression> equalCompensateConjunctions = new ArrayList<>();
@ -276,6 +405,7 @@ public abstract class AbstractMaterializedViewRule {
}
if (queryEquivalenceClass.isEmpty()
&& !viewEquivalenceClass.isEmpty()) {
logger.info(currentClassName + " view has equivalence class but query not so return empty");
return SplitPredicate.empty();
}
EquivalenceClassSetMapping queryToViewEquivalenceMapping =
@ -283,6 +413,7 @@ public abstract class AbstractMaterializedViewRule {
// can not map all target equivalence class, can not compensate
if (queryToViewEquivalenceMapping.getEquivalenceClassSetMap().size()
< viewEquivalenceClass.getEquivalenceSetList().size()) {
logger.info(currentClassName + " view has more equivalence than query so return empty");
return SplitPredicate.empty();
}
// do equal compensate
@ -330,6 +461,7 @@ public abstract class AbstractMaterializedViewRule {
// query range predicate can not contain all view range predicate when view have range predicate, bail out
if (!viewRangePredicateQueryBased.equals(BooleanLiteral.TRUE)
&& !queryRangeSet.containsAll(viewRangeQueryBasedSet)) {
logger.info(currentClassName + " query range predicate set can not contains all view range predicate");
return SplitPredicate.empty();
}
queryRangeSet.removeAll(viewRangeQueryBasedSet);
@ -349,6 +481,8 @@ public abstract class AbstractMaterializedViewRule {
// bail out
if (!viewResidualPredicateQueryBased.equals(BooleanLiteral.TRUE)
&& !queryResidualSet.containsAll(viewResidualQueryBasedSet)) {
logger.info(
currentClassName + " query residual predicate set can not contains all view residual predicate");
return SplitPredicate.empty();
}
queryResidualSet.removeAll(viewResidualQueryBasedSet);

View File

@ -19,12 +19,8 @@ package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.MTMV;
import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.TableIf;
import org.apache.doris.catalog.TableIf.TableType;
import org.apache.doris.common.MetaNotFoundException;
import org.apache.doris.mtmv.BaseTableInfo;
import org.apache.doris.mtmv.MTMVRelationManager;
import org.apache.doris.nereids.CascadesContext;
import org.apache.doris.nereids.NereidsPlanner;
import org.apache.doris.nereids.PlannerHook;
@ -42,13 +38,14 @@ import com.google.common.collect.Sets;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
/** If enable query rewrite with mv, should init materialization context after analyze*/
/**
* If enable query rewrite with mv, should init materialization context after analyze
*/
public class InitMaterializationContextHook implements PlannerHook {
public static final Logger LOG = LogManager.getLogger(InitMaterializationContextHook.class);
@ -60,7 +57,9 @@ public class InitMaterializationContextHook implements PlannerHook {
}
private void initMaterializationContext(CascadesContext cascadesContext) {
if (!cascadesContext.getConnectContext().getSessionVariable().isEnableMaterializedViewRewrite()) {
return;
}
Plan rewritePlan = cascadesContext.getRewritePlan();
TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet());
rewritePlan.accept(TableCollector.INSTANCE, collectorContext);
@ -68,48 +67,30 @@ public class InitMaterializationContextHook implements PlannerHook {
if (collectedTables.isEmpty()) {
return;
}
List<BaseTableInfo> baseTableUsed =
List<BaseTableInfo> usedBaseTables =
collectedTables.stream().map(BaseTableInfo::new).collect(Collectors.toList());
// TODO the logic should be move to MTMVRelationManager later when getAvailableMaterializedView is ready in
// MV Cache manager
Env env = cascadesContext.getConnectContext().getEnv();
MTMVRelationManager cacheManager = env.getMtmvService().getRelationManager();
Set<BaseTableInfo> materializedViews = new HashSet<>();
for (BaseTableInfo baseTableInfo : baseTableUsed) {
Set<BaseTableInfo> mtmvsByBaseTable = cacheManager.getMtmvsByBaseTable(baseTableInfo);
if (mtmvsByBaseTable == null || mtmvsByBaseTable.isEmpty()) {
continue;
}
materializedViews.addAll(mtmvsByBaseTable);
}
if (materializedViews.isEmpty()) {
Set<MTMV> availableMTMVs = Env.getCurrentEnv().getMtmvService().getRelationManager()
.getAvailableMTMVs(usedBaseTables);
if (availableMTMVs.isEmpty()) {
return;
}
materializedViews.forEach(mvBaseTableInfo -> {
try {
MTMV materializedView = (MTMV) Env.getCurrentInternalCatalog()
.getDbOrMetaException(mvBaseTableInfo.getDbId())
.getTableOrMetaException(mvBaseTableInfo.getTableId(), TableType.MATERIALIZED_VIEW);
// generate outside, maybe add partition filter in the future
LogicalOlapScan mvScan = new LogicalOlapScan(
cascadesContext.getStatementContext().getNextRelationId(),
(OlapTable) materializedView,
ImmutableList.of(materializedView.getQualifiedDbName()),
// this must be empty, or it will be used to sample
Lists.newArrayList(),
Lists.newArrayList(),
Optional.empty());
mvScan = mvScan.withMaterializedIndexSelected(PreAggStatus.on(), materializedView.getBaseIndexId());
List<NamedExpression> mvProjects = mvScan.getOutput().stream().map(NamedExpression.class::cast)
.collect(Collectors.toList());
// todo should force keep consistency to mv sql plan output
Plan projectScan = new LogicalProject<Plan>(mvProjects, mvScan);
cascadesContext.addMaterializationContext(
MaterializationContext.fromMaterializedView(materializedView, projectScan, cascadesContext));
} catch (MetaNotFoundException metaNotFoundException) {
LOG.error(mvBaseTableInfo.toString() + " can not find corresponding materialized view.");
}
availableMTMVs.forEach(materializedView -> {
// generate outside, maybe add partition filter in the future
LogicalOlapScan mvScan = new LogicalOlapScan(
cascadesContext.getStatementContext().getNextRelationId(),
materializedView,
ImmutableList.of(materializedView.getQualifiedDbName()),
// this must be empty, or it will be used to sample
Lists.newArrayList(),
Lists.newArrayList(),
Optional.empty());
mvScan = mvScan.withMaterializedIndexSelected(PreAggStatus.on(), materializedView.getBaseIndexId());
List<NamedExpression> mvProjects = mvScan.getOutput().stream().map(NamedExpression.class::cast)
.collect(Collectors.toList());
// todo should force keep consistency to mv sql plan output
Plan projectScan = new LogicalProject<Plan>(mvProjects, mvScan);
cascadesContext.addMaterializationContext(
MaterializationContext.fromMaterializedView(materializedView, projectScan, cascadesContext));
});
}
}

View File

@ -19,6 +19,7 @@ package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.catalog.MTMV;
import org.apache.doris.catalog.Table;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.mtmv.MTMVCache;
import org.apache.doris.nereids.CascadesContext;
import org.apache.doris.nereids.memo.GroupId;
@ -27,6 +28,8 @@ import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.util.ExpressionUtils;
import com.google.common.collect.ImmutableList;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.HashSet;
import java.util.List;
@ -37,6 +40,7 @@ import java.util.Set;
*/
public class MaterializationContext {
private static final Logger LOG = LogManager.getLogger(MaterializationContext.class);
private MTMV mtmv;
// Should use stmt id generator in query context
private final Plan mvScanPlan;
@ -46,6 +50,7 @@ public class MaterializationContext {
private final Set<GroupId> matchedGroups = new HashSet<>();
// generate form mv scan plan
private ExpressionMapping mvExprToMvScanExprMapping;
private boolean available = true;
/**
* MaterializationContext, this contains necessary info for query rewriting by mv
@ -59,11 +64,16 @@ public class MaterializationContext {
this.mvScanPlan = mvScanPlan;
this.baseTables = baseTables;
this.baseViews = baseViews;
MTMVCache mtmvCache = mtmv.getCache();
// TODO This logic should move to materialized view cache manager
MTMVCache mtmvCache = null;
try {
mtmvCache = mtmv.getOrGenerateCache(cascadesContext.getConnectContext());
} catch (AnalysisException e) {
LOG.warn("MaterializationContext init mv cache generate fail", e);
}
if (mtmvCache == null) {
mtmvCache = mtmvCache.from(mtmv, cascadesContext.getConnectContext());
mtmv.setCache(mtmvCache);
this.available = false;
return;
}
// mv output expression shuttle, this will be used to expression rewrite
this.mvExprToMvScanExprMapping = ExpressionMapping.generate(
@ -85,7 +95,7 @@ public class MaterializationContext {
matchedGroups.add(groupId);
}
public MTMV getMtmv() {
public MTMV getMTMV() {
return mtmv;
}
@ -105,6 +115,10 @@ public class MaterializationContext {
return mvExprToMvScanExprMapping;
}
public boolean isAvailable() {
return available;
}
/**
* MaterializationContext fromMaterializedView
*/

View File

@ -18,9 +18,7 @@
package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RulePromise;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
@ -31,7 +29,7 @@ import java.util.List;
/**
* This is responsible for aggregate rewriting according to different pattern
* */
public class MaterializedViewAggregateRule extends AbstractMaterializedViewAggregateRule implements RewriteRuleFactory {
public class MaterializedViewAggregateRule extends AbstractMaterializedViewAggregateRule {
public static final MaterializedViewAggregateRule INSTANCE = new MaterializedViewAggregateRule();
@ -41,6 +39,6 @@ public class MaterializedViewAggregateRule extends AbstractMaterializedViewAggre
logicalAggregate(any()).thenApplyMulti(ctx -> {
LogicalAggregate<Plan> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_ONLY_AGGREGATE, RulePromise.EXPLORE));
}).toRule(RuleType.MATERIALIZED_VIEW_ONLY_AGGREGATE));
}
}

View File

@ -0,0 +1,45 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* This is responsible for join pattern such as filter on join
*/
public class MaterializedViewFilterJoinRule extends AbstractMaterializedViewJoinRule {
public static final MaterializedViewFilterJoinRule INSTANCE = new MaterializedViewFilterJoinRule();
@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalFilter(logicalJoin(any(), any())).thenApplyMulti(ctx -> {
LogicalFilter<LogicalJoin<Plan, Plan>> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_FILTER_JOIN));
}
}

View File

@ -0,0 +1,46 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* This is responsible for join pattern such as filter on project on join
*/
public class MaterializedViewFilterProjectJoinRule extends AbstractMaterializedViewJoinRule {
public static final MaterializedViewFilterProjectJoinRule INSTANCE = new MaterializedViewFilterProjectJoinRule();
@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalFilter(logicalProject(logicalJoin(any(), any()))).thenApplyMulti(ctx -> {
LogicalFilter<LogicalProject<LogicalJoin<Plan, Plan>>> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_FILTER_PROJECT_JOIN));
}
}

View File

@ -0,0 +1,44 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* This is responsible for join pattern such as only join
*/
public class MaterializedViewOnlyJoinRule extends AbstractMaterializedViewJoinRule {
public static final MaterializedViewOnlyJoinRule INSTANCE = new MaterializedViewOnlyJoinRule();
@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalJoin(any(), any()).thenApplyMulti(ctx -> {
LogicalJoin<Plan, Plan> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_ONLY_JOIN));
}
}

View File

@ -18,9 +18,7 @@
package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RulePromise;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory;
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;
@ -30,8 +28,7 @@ import com.google.common.collect.ImmutableList;
import java.util.List;
/**MaterializedViewProjectAggregateRule*/
public class MaterializedViewProjectAggregateRule extends AbstractMaterializedViewAggregateRule implements
RewriteRuleFactory {
public class MaterializedViewProjectAggregateRule extends AbstractMaterializedViewAggregateRule {
public static final MaterializedViewProjectAggregateRule INSTANCE = new MaterializedViewProjectAggregateRule();
@ -41,6 +38,6 @@ public class MaterializedViewProjectAggregateRule extends AbstractMaterializedVi
logicalProject(logicalAggregate(any())).thenApplyMulti(ctx -> {
LogicalProject<LogicalAggregate<Plan>> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_AGGREGATE, RulePromise.EXPLORE));
}).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_AGGREGATE));
}
}

View File

@ -0,0 +1,46 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* This is responsible for join pattern such as project on filter on join
*/
public class MaterializedViewProjectFilterJoinRule extends AbstractMaterializedViewJoinRule {
public static final MaterializedViewProjectFilterJoinRule INSTANCE = new MaterializedViewProjectFilterJoinRule();
@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalProject(logicalFilter(logicalJoin(any(), any()))).thenApplyMulti(ctx -> {
LogicalProject<LogicalFilter<LogicalJoin<Plan, Plan>>> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_FILTER_JOIN));
}
}

View File

@ -18,9 +18,7 @@
package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RulePromise;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
@ -30,9 +28,9 @@ import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* This is responsible for join rewriting according to different pattern
* This is responsible for join pattern such as project on join
* */
public class MaterializedViewProjectJoinRule extends AbstractMaterializedViewJoinRule implements RewriteRuleFactory {
public class MaterializedViewProjectJoinRule extends AbstractMaterializedViewJoinRule {
public static final MaterializedViewProjectJoinRule INSTANCE = new MaterializedViewProjectJoinRule();
@ -42,6 +40,6 @@ public class MaterializedViewProjectJoinRule extends AbstractMaterializedViewJoi
logicalProject(logicalJoin(any(), any())).thenApplyMulti(ctx -> {
LogicalProject<LogicalJoin<Plan, Plan>> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_JOIN, RulePromise.EXPLORE));
}).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_JOIN));
}
}

View File

@ -18,14 +18,13 @@
package org.apache.doris.nereids.rules.exploration.mv;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory;
import java.util.List;
/**
* This is responsible for single table rewriting according to different pattern
* */
public class MaterializedViewScanRule extends AbstractMaterializedViewRule implements RewriteRuleFactory {
public class MaterializedViewScanRule extends AbstractMaterializedViewRule {
@Override
public List<Rule> buildRules() {

View File

@ -282,7 +282,9 @@ public class StructInfo {
@Override
public Void visit(Plan plan, Set<Expression> predicates) {
// Just collect the filter in top plan, if meet other node except project and filter, return
if (!(plan instanceof LogicalProject) && !(plan instanceof LogicalFilter)) {
if (!(plan instanceof LogicalProject)
&& !(plan instanceof LogicalFilter)
&& !(plan instanceof LogicalAggregate)) {
return null;
}
if (plan instanceof LogicalFilter) {
@ -396,7 +398,7 @@ public class StructInfo {
super.visit(aggregate, context);
return true;
}
if (plan instanceof LogicalProject) {
if (plan instanceof LogicalProject || plan instanceof LogicalFilter) {
super.visit(plan, context);
return true;
}

View File

@ -0,0 +1,67 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions;
import org.apache.doris.nereids.trees.TreeNode;
import org.apache.doris.nereids.trees.expressions.shape.LeafExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
import com.google.common.collect.ImmutableList;
import java.util.List;
/**
* This represents any expression, it means it equals any expression
*/
public class Any extends Expression implements LeafExpression {
public static final Any INSTANCE = new Any(ImmutableList.of());
private Any(Expression... children) {
super(children);
}
private Any(List<Expression> children) {
super(children);
}
@Override
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitAny(this, context);
}
@Override
public boolean nullable() {
return false;
}
@Override
public boolean equals(Object o) {
return true;
}
@Override
public int hashCode() {
return 0;
}
@Override
public boolean deepEquals(TreeNode<?> that) {
return true;
}
}

View File

@ -77,10 +77,6 @@ public abstract class AggregateFunction extends BoundFunction implements Expects
return distinct;
}
public Class<? extends AggregateFunction> getRollup() {
return null;
}
@Override
public boolean equals(Object o) {
if (this == o) {

View File

@ -0,0 +1,32 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package org.apache.doris.nereids.trees.expressions.functions.agg;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.Function;
/**
* Could roll up trait, if a function could roll up in aggregate, it will implement the interface
*/
public interface CouldRollUp {
/**
* construct the roll up function with custom param
*/
Function constructRollUp(Expression param, Expression... varParams);
}

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable;
import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
import org.apache.doris.nereids.trees.expressions.functions.Function;
import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic;
import org.apache.doris.nereids.trees.expressions.literal.Literal;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
@ -36,7 +37,7 @@ import java.util.List;
/** count agg function. */
public class Count extends AggregateFunction
implements ExplicitlyCastableSignature, AlwaysNotNullable, SupportWindowAnalytic {
implements ExplicitlyCastableSignature, AlwaysNotNullable, SupportWindowAnalytic, CouldRollUp {
public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
// count(*)
@ -142,4 +143,13 @@ public class Count extends AggregateFunction
public List<FunctionSignature> getSignatures() {
return SIGNATURES;
}
@Override
public Function constructRollUp(Expression param, Expression... varParams) {
if (this.isDistinct()) {
return new BitmapUnionCount(param);
} else {
return new Sum(param);
}
}
}

View File

@ -22,6 +22,7 @@ import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.CustomSignature;
import org.apache.doris.nereids.trees.expressions.functions.Function;
import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
@ -34,7 +35,7 @@ import java.util.List;
/** max agg function. */
public class Max extends NullableAggregateFunction
implements UnaryExpression, CustomSignature, SupportWindowAnalytic {
implements UnaryExpression, CustomSignature, SupportWindowAnalytic, CouldRollUp {
public Max(Expression child) {
this(false, false, child);
}
@ -80,4 +81,9 @@ public class Max extends NullableAggregateFunction
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitMax(this, context);
}
@Override
public Function constructRollUp(Expression param, Expression... varParams) {
return new Max(this.distinct, this.alwaysNullable, param);
}
}

View File

@ -22,6 +22,7 @@ import org.apache.doris.catalog.Type;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.CustomSignature;
import org.apache.doris.nereids.trees.expressions.functions.Function;
import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
@ -34,7 +35,7 @@ import java.util.List;
/** min agg function. */
public class Min extends NullableAggregateFunction
implements UnaryExpression, CustomSignature, SupportWindowAnalytic {
implements UnaryExpression, CustomSignature, SupportWindowAnalytic, CouldRollUp {
public Min(Expression child) {
this(false, false, child);
@ -81,4 +82,9 @@ public class Min extends NullableAggregateFunction
public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
return visitor.visitMin(this, context);
}
@Override
public Function constructRollUp(Expression param, Expression... varParams) {
return new Min(this.distinct, this.alwaysNullable, param);
}
}

View File

@ -22,6 +22,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.functions.ComputePrecisionForSum;
import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
import org.apache.doris.nereids.trees.expressions.functions.Function;
import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic;
import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression;
import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
@ -45,7 +46,8 @@ import java.util.List;
* AggregateFunction 'sum'. This class is generated by GenerateFunction.
*/
public class Sum extends NullableAggregateFunction
implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecisionForSum, SupportWindowAnalytic {
implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecisionForSum, SupportWindowAnalytic,
CouldRollUp {
public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
FunctionSignature.ret(BigIntType.INSTANCE).args(BooleanType.INSTANCE),
@ -111,7 +113,7 @@ public class Sum extends NullableAggregateFunction
}
@Override
public Class<? extends AggregateFunction> getRollup() {
return Sum.class;
public Function constructRollUp(Expression param, Expression... varParams) {
return new Sum(this.distinct, param);
}
}

View File

@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.Add;
import org.apache.doris.nereids.trees.expressions.AggregateExpression;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.And;
import org.apache.doris.nereids.trees.expressions.Any;
import org.apache.doris.nereids.trees.expressions.ArrayItemReference;
import org.apache.doris.nereids.trees.expressions.AssertNumRowsElement;
import org.apache.doris.nereids.trees.expressions.BinaryArithmetic;
@ -499,6 +500,10 @@ public abstract class ExpressionVisitor<R, C>
return visitMatch(matchPhrasePrefix, context);
}
public R visitAny(Any any, C context) {
return visit(any, context);
}
/* ********************************************************************************************
* Unbound expressions
* ********************************************************************************************/

View File

@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisit
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer.ExpressionReplaceContext;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import java.util.Map;
@ -66,9 +67,28 @@ public class ExpressionLineageReplacer extends DefaultPlanVisitor<Expression, Ex
public Expression visitNamedExpression(NamedExpression namedExpression,
Map<ExprId, Expression> exprIdExpressionMap) {
if (exprIdExpressionMap.containsKey(namedExpression.getExprId())) {
return super.visit(exprIdExpressionMap.get(namedExpression.getExprId()), exprIdExpressionMap);
return visit(exprIdExpressionMap.get(namedExpression.getExprId()), exprIdExpressionMap);
}
return super.visitNamedExpression(namedExpression, exprIdExpressionMap);
return visit(namedExpression, exprIdExpressionMap);
}
@Override
public Expression visit(Expression expr, Map<ExprId, Expression> exprIdExpressionMap) {
if (expr instanceof NamedExpression
&& expr.arity() == 0
&& exprIdExpressionMap.containsKey(((NamedExpression) expr).getExprId())) {
expr = exprIdExpressionMap.get(((NamedExpression) expr).getExprId());
}
List<Expression> newChildren = new ArrayList<>(expr.arity());
boolean hasNewChildren = false;
for (Expression child : expr.children()) {
Expression newChild = child.accept(this, exprIdExpressionMap);
if (newChild != child) {
hasNewChildren = true;
}
newChildren.add(newChild);
}
return hasNewChildren ? expr.withChildren(newChildren) : expr;
}
}

View File

@ -362,12 +362,7 @@ public class ExpressionUtils {
@Override
public Expression visit(Expression expr, Map<? extends Expression, ? extends Expression> replaceMap) {
if (replaceMap.containsKey(expr)) {
Expression replacedExpression = replaceMap.get(expr);
if (replacedExpression instanceof SlotReference
&& replacedExpression.nullable() != expr.nullable()) {
replacedExpression = ((SlotReference) replacedExpression).withNullable(expr.nullable());
}
return replacedExpression;
return replaceMap.get(expr);
}
return super.visit(expr, replaceMap);
}

View File

@ -122,7 +122,7 @@ public class MaterializedViewUtilsTest extends TestWithFeService {
}
@Test
public void getRelatedTableInfoTestWithAliasAndGroupTest() {
public void getRelatedTableInfoTestWithSubqueryTest() {
PlanChecker.from(connectContext)
.checkExplain("SELECT l.L_SHIPDATE AS ship_data_alias, o.O_ORDERDATE, count(*) "
+ "FROM "
@ -147,6 +147,36 @@ public class MaterializedViewUtilsTest extends TestWithFeService {
});
}
@Test
public void getRelatedTableInfoTestWithAliasAndGroupTest() {
PlanChecker.from(connectContext)
.checkExplain("SELECT t1.L_SHIPDATE, t2.O_ORDERDATE, t1.L_QUANTITY, t2.O_ORDERSTATUS, "
+ "count(distinct case when t1.L_SUPPKEY > 0 then t2.O_ORDERSTATUS else null end) as cnt_1 "
+ "from "
+ " (select * from "
+ " lineitem "
+ " where L_SHIPDATE in ('2017-01-30')) t1 "
+ "left join "
+ " (select * from "
+ " orders "
+ " where O_ORDERDATE in ('2017-01-30')) t2 "
+ "on t1.L_ORDERKEY = t2.O_ORDERKEY "
+ "group by "
+ "t1.L_SHIPDATE, "
+ "t2.O_ORDERDATE, "
+ "t1.L_QUANTITY, "
+ "t2.O_ORDERSTATUS;",
nereidsPlanner -> {
Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan();
Optional<RelatedTableInfo> relatedTableInfo =
MaterializedViewUtils.getRelatedTableInfo("L_SHIPDATE", rewrittenPlan);
checkRelatedTableInfo(relatedTableInfo,
"lineitem",
"L_SHIPDATE",
true);
});
}
@Test
public void getRelatedTableInfoTestWithoutPartitionTest() {
PlanChecker.from(connectContext)

View File

@ -0,0 +1,123 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !query13_0_before --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query13_0_after --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query14_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 \N \N \N 1 0
2 4 2023-12-10 \N \N \N 1 0
3 3 2023-12-11 \N \N \N 1 0
4 3 2023-12-09 \N \N \N 1 0
-- !query14_0_after --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 \N \N \N 1 0
2 4 2023-12-10 \N \N \N 1 0
3 3 2023-12-11 \N \N \N 1 0
4 3 2023-12-09 \N \N \N 1 0
-- !query15_0_before --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query15_0_after --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query16_0_before --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query16_0_after --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query17_0_before --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query17_0_after --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query18_0_before --
-- !query18_0_after --
-- !query19_0_before --
2 3 2023-12-08 20.00
2 3 2023-12-12 57.40
2 4 2023-12-10 46.00
-- !query19_0_after --
2 3 2023-12-08 20.00
2 3 2023-12-12 57.40
2 4 2023-12-10 46.00
-- !query20_0_before --
2023-12-08 3 2023-12-08 20.00 10.50 9.50 2 0
2023-12-09 3 2023-12-09 11.50 11.50 11.50 1 0
2023-12-10 4 2023-12-10 46.00 33.50 12.50 2 0
2023-12-11 3 2023-12-11 43.20 43.20 43.20 1 0
2023-12-12 3 2023-12-12 57.40 56.20 1.20 2 0
-- !query20_0_after --
2023-12-08 3 2023-12-08 20.00 10.50 9.50 2 0
2023-12-09 3 2023-12-09 11.50 11.50 11.50 1 0
2023-12-10 4 2023-12-10 46.00 33.50 12.50 2 0
2023-12-11 3 2023-12-11 43.20 43.20 43.20 1 0
2023-12-12 3 2023-12-12 57.40 56.20 1.20 2 0
-- !query21_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
-- !query21_0_after --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
-- !query22_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
3 3 2023-12-11 43.20 43.20 43.20 1 0
4 3 2023-12-09 11.50 11.50 11.50 1 0
-- !query22_0_after --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
3 3 2023-12-11 43.20 43.20 43.20 1 0
4 3 2023-12-09 11.50 11.50 11.50 1 0
-- !query23_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2 0
-- !query23_0_after --
2 3 2023-12-08 20.00 10.50 9.50 2 0
-- !query24_0_before --
3 2023-12-08 20.00 10.50 9.50 2 0
3 2023-12-09 11.50 11.50 11.50 1 0
3 2023-12-11 43.20 43.20 43.20 1 0
3 2023-12-12 57.40 56.20 1.20 2 0
-- !query24_0_after --
3 2023-12-08 20.00 10.50 9.50 2 0
3 2023-12-09 11.50 11.50 11.50 1 0
3 2023-12-11 43.20 43.20 43.20 1 0
3 2023-12-12 57.40 56.20 1.20 2 0
-- !query25_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2
2 3 2023-12-12 57.40 56.20 1.20 2
2 4 2023-12-10 46.00 33.50 12.50 2
3 3 2023-12-11 43.20 43.20 43.20 1
4 3 2023-12-09 11.50 11.50 11.50 1
-- !query25_0_after --
2 3 2023-12-08 20.00 10.50 9.50 2
2 3 2023-12-12 57.40 56.20 1.20 2
2 4 2023-12-10 46.00 33.50 12.50 2
3 3 2023-12-11 43.20 43.20 43.20 1
4 3 2023-12-09 11.50 11.50 11.50 1

View File

@ -0,0 +1,127 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !query1_0_before --
-- !query1_0_after --
1 yy 0 0 77.50 33.50 9.50 5
2 mi 0 0 57.40 56.20 1.20 2
2 mm 0 0 43.20 43.20 43.20 1
-- !query1_2_before --
1 yy 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
2 mi 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
2 mm 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
-- !query1_2_after --
1 yy 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
2 mi 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
2 mm 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
-- !query2_0_before --
1 yy 0 0 0 0 0 0 0 0 0 0 0
2 mi 0 0 0 0 0 0 0 0 0 0 0
2 mm 0 0 0 0 0 0 0 0 0 0 0
-- !query2_0_after --
1 yy 0 0 0 0 0 0 0 0 0 0 0
2 mi 0 0 0 0 0 0 0 0 0 0 0
2 mm 0 0 0 0 0 0 0 0 0 0 0
-- !query3_0_before --
0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
-- !query3_0_after --
0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
-- !query13_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
-- !query13_0_after --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
-- !query14_0_before --
2 3 \N \N \N \N 1 0
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 4 \N \N \N \N 1 0
3 3 \N \N \N \N 1 0
4 3 \N \N \N \N 1 0
-- !query14_0_after --
2 3 \N \N \N \N 1 0
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 4 \N \N \N \N 1 0
3 3 \N \N \N \N 1 0
4 3 \N \N \N \N 1 0
-- !query15_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query15_0_after --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
3 3 2023-12-11 43.20 43.20 43.20 1 0
4 3 2023-12-09 11.50 11.50 11.50 1 0
-- !query16_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
3 3 2023-12-11 43.20 43.20 43.20 1 0
4 3 2023-12-09 11.50 11.50 11.50 1 0
-- !query16_0_after --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
2 4 2023-12-10 46.00 33.50 12.50 2 0
3 3 2023-12-11 43.20 43.20 43.20 1 0
4 3 2023-12-09 11.50 11.50 11.50 1 0
-- !query17_0_before --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query17_0_after --
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query18_0_before --
-- !query18_0_after --
-- !query18_1_before --
-- !query18_1_after --
-- !query18_2_before --
-- !query18_2_after --
-- !query19_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2
2 3 2023-12-12 57.40 56.20 1.20 2
2 4 2023-12-10 46.00 33.50 12.50 2
3 3 2023-12-11 43.20 43.20 43.20 1
4 3 2023-12-09 11.50 11.50 11.50 1
-- !query19_0_after --
2 3 2023-12-08 20.00 10.50 9.50 2
2 3 2023-12-12 57.40 56.20 1.20 2
2 4 2023-12-10 46.00 33.50 12.50 2
3 3 2023-12-11 43.20 43.20 43.20 1
4 3 2023-12-09 11.50 11.50 11.50 1
-- !query19_1_before --
4 1 77.50
4 2 43.20
6 2 57.40
-- !query19_1_after --
4 1 77.50
4 2 43.20
6 2 57.40

View File

@ -0,0 +1,267 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !query1_0_before --
4
4
4
4
4
4
6
6
-- !query1_0_after --
4
4
4
4
4
4
6
6
-- !query1_1_before --
4
4
4
4
6
6
6
6
-- !query1_1_after --
4
4
4
4
6
6
6
6
-- !query1_2_before --
4
4
4
4
4
4
6
6
-- !query1_2_after --
4
4
4
4
4
4
6
6
-- !query1_3_before --
1 1
1 1
1 1
1 1
1 1
2 2
2 2
2 2
-- !query1_3_after --
1 1
1 1
1 1
1 1
1 1
2 2
2 2
2 2
-- !query1_4_before --
4
4
4
4
6
6
6
6
-- !query1_4_after --
4
4
4
4
6
6
6
6
-- !query2_0_before --
-- !query2_0_after --
-- !query2_1_before --
4
4
4
4
4
4
6
6
-- !query2_1_after --
4
4
4
4
4
4
6
6
-- !query2_3_before --
4
4
4
4
6
6
6
6
-- !query2_3_after --
4
4
4
4
6
6
6
6
-- !query3_0_before --
4
4
4
4
4
4
6
6
-- !query3_0_after --
4
4
4
4
4
4
6
6
-- !query3_1_before --
4
4
4
4
4
4
6
6
-- !query3_1_after --
4
4
4
4
4
4
6
6
-- !query3_2_before --
4
4
4
4
4
4
6
6
-- !query3_2_after --
4
4
4
4
4
4
6
6
-- !query3_3_before --
4
4
4
4
6
6
6
6
-- !query3_3_after --
4
4
4
4
6
6
6
6
-- !query4_0_before --
4
4
4
4
4
4
-- !query4_0_after --
4
4
4
4
4
4
6
6
-- !query5_0_before --
4
4
4
4
4
4
6
6
-- !query5_0_after --
4
4
4
4
4
4
6
6
-- !query10_0_before --
-- !query10_0_after --

View File

@ -0,0 +1,223 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !query1_0_before --
4
4
4
4
4
4
6
6
-- !query1_0_after --
4
4
4
4
4
4
6
6
-- !query1_1_before --
4
4
4
4
4
4
4
4
6
6
6
6
-- !query1_1_after --
4
4
4
4
4
4
4
4
6
6
6
6
-- !query1_2_before --
4
4
4
4
4
4
6
6
-- !query1_2_after --
4
4
4
4
4
4
6
6
-- !query1_3_before --
1 1
1 1
1 1
1 1
1 1
2 2
2 2
2 2
-- !query1_3_after --
1 1
1 1
1 1
1 1
1 1
2 2
2 2
2 2
-- !query2_0_before --
-- !query2_0_after --
-- !query2_1_before --
4
4
4
4
4
4
6
6
-- !query2_1_after --
4
4
4
4
4
4
6
6
-- !query3_0_before --
4
4
4
4
4
4
6
6
-- !query3_0_after --
4
4
4
4
4
4
6
6
-- !query3_1_before --
4
4
4
4
4
4
6
6
-- !query3_1_after --
4
4
4
4
4
4
6
6
-- !query3_2_before --
4
4
4
4
4
4
6
6
-- !query3_2_after --
4
4
4
4
4
4
6
6
-- !query4_0_before --
4
4
-- !query4_0_after --
4
4
-- !query5_0_before --
4
4
4
4
4
4
6
6
-- !query5_0_after --
4
4
4
4
4
4
6
6
-- !query6_0_before --
2 3 2023-12-08
2 3 2023-12-08
2 3 2023-12-12
2 4 2023-12-10
3 3 2023-12-11
4 3 2023-12-09
-- !query6_0_after --
2 3 2023-12-08
2 3 2023-12-08
2 3 2023-12-12
2 4 2023-12-10
3 3 2023-12-11
4 3 2023-12-09
-- !query7_0_before --
3 3 2023-12-11
-- !query7_0_after --
3 3 2023-12-11

View File

@ -22,6 +22,8 @@ suite("aggregate_with_roll_up") {
sql "SET enable_fallback_to_original_planner=false"
sql "SET enable_materialized_view_rewrite=true"
sql "SET enable_nereids_timeout = false"
// tmp disable to rewrite, will be removed in the future
sql "SET disable_nereids_rules = 'INFER_PREDICATES, ELIMINATE_OUTER_JOIN'"
sql """
drop table if exists orders
@ -29,18 +31,19 @@ suite("aggregate_with_roll_up") {
sql """
CREATE TABLE IF NOT EXISTS orders (
O_ORDERKEY INTEGER NOT NULL,
O_CUSTKEY INTEGER NOT NULL,
O_ORDERSTATUS CHAR(1) NOT NULL,
O_TOTALPRICE DECIMALV3(15,2) NOT NULL,
O_ORDERDATE DATE NOT NULL,
O_ORDERPRIORITY CHAR(15) NOT NULL,
O_CLERK CHAR(15) NOT NULL,
O_SHIPPRIORITY INTEGER NOT NULL,
o_orderkey INTEGER NOT NULL,
o_custkey INTEGER NOT NULL,
o_orderstatus CHAR(1) NOT NULL,
o_totalprice DECIMALV3(15,2) NOT NULL,
o_orderdate DATE NOT NULL,
o_orderpriority CHAR(15) NOT NULL,
o_clerk CHAR(15) NOT NULL,
o_shippriority INTEGER NOT NULL,
O_COMMENT VARCHAR(79) NOT NULL
)
DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY)
DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3
DUPLICATE KEY(o_orderkey, o_custkey)
PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30'))
DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
@ -52,25 +55,26 @@ suite("aggregate_with_roll_up") {
sql"""
CREATE TABLE IF NOT EXISTS lineitem (
L_ORDERKEY INTEGER NOT NULL,
L_PARTKEY INTEGER NOT NULL,
L_SUPPKEY INTEGER NOT NULL,
L_LINENUMBER INTEGER NOT NULL,
L_QUANTITY DECIMALV3(15,2) NOT NULL,
L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL,
L_DISCOUNT DECIMALV3(15,2) NOT NULL,
L_TAX DECIMALV3(15,2) NOT NULL,
L_RETURNFLAG CHAR(1) NOT NULL,
L_LINESTATUS CHAR(1) NOT NULL,
L_SHIPDATE DATE NOT NULL,
L_COMMITDATE DATE NOT NULL,
L_RECEIPTDATE DATE NOT NULL,
L_SHIPINSTRUCT CHAR(25) NOT NULL,
L_SHIPMODE CHAR(10) NOT NULL,
L_COMMENT VARCHAR(44) NOT NULL
l_orderkey INTEGER NOT NULL,
l_partkey INTEGER NOT NULL,
l_suppkey INTEGER NOT NULL,
l_linenumber INTEGER NOT NULL,
l_quantity DECIMALV3(15,2) NOT NULL,
l_extendedprice DECIMALV3(15,2) NOT NULL,
l_discount DECIMALV3(15,2) NOT NULL,
l_tax DECIMALV3(15,2) NOT NULL,
l_returnflag CHAR(1) NOT NULL,
l_linestatus CHAR(1) NOT NULL,
l_shipdate DATE NOT NULL,
l_commitdate DATE NOT NULL,
l_receiptdate DATE NOT NULL,
l_shipinstruct CHAR(25) NOT NULL,
l_shipmode CHAR(10) NOT NULL,
l_comment VARCHAR(44) NOT NULL
)
DUPLICATE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER)
DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3
DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber)
PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30'))
DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
@ -82,33 +86,39 @@ suite("aggregate_with_roll_up") {
sql """
CREATE TABLE IF NOT EXISTS partsupp (
PS_PARTKEY INTEGER NOT NULL,
PS_SUPPKEY INTEGER NOT NULL,
PS_AVAILQTY INTEGER NOT NULL,
PS_SUPPLYCOST DECIMALV3(15,2) NOT NULL,
PS_COMMENT VARCHAR(199) NOT NULL
ps_partkey INTEGER NOT NULL,
ps_suppkey INTEGER NOT NULL,
ps_availqty INTEGER NOT NULL,
ps_supplycost DECIMALV3(15,2) NOT NULL,
ps_comment VARCHAR(199) NOT NULL
)
DUPLICATE KEY(PS_PARTKEY, PS_SUPPKEY)
DISTRIBUTED BY HASH(PS_PARTKEY) BUCKETS 3
DUPLICATE KEY(ps_partkey, ps_suppkey)
DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
"""
waiteCreateTableFinished("lineitem")
sql """ insert into lineitem values
(1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(2, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-11', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');"""
waiteCreateTableFinished("orders")
sql """
insert into orders values
(1, 1, 'ok', 99.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(2, 2, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm'),
(2, 3, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm');
(2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');
"""
sql """
insert into orders values
(1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'),
(3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'),
(3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'),
(4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'),
(5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'),
(5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi');
"""
waiteCreateTableFinished("partsupp")
sql """
insert into partsupp values
(2, 3, 9, 10.01, 'supply1'),
@ -134,20 +144,571 @@ suite("aggregate_with_roll_up") {
}
}
// select + from + inner join + group by
def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, sum(O_TOTALPRICE) as sum_alias " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"group by lineitem.L_LINENUMBER, orders.O_CUSTKEY "
def query1_0 = "select lineitem.L_LINENUMBER, sum(O_TOTALPRICE) as sum_alias " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"group by lineitem.L_LINENUMBER"
// query
// fix later
// order_qt_query1_0_before "${query1_0}"
check_rewrite(mv1_0, query1_0, "mv1_0")
// order_qt_query1_0_after "${query1_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0"""
}
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})"
}
}
// single table
// filter + use roll up dimension
def mv1_1 = "select o_orderdate, o_shippriority, o_comment, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, " +
"bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 " +
"from orders " +
"group by " +
"o_orderdate, " +
"o_shippriority, " +
"o_comment "
def query1_1 = "select o_shippriority, o_comment, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*) " +
"from orders " +
"where o_orderdate = '2023-12-09' " +
"group by " +
"o_shippriority, " +
"o_comment "
// rewrite success but cbo not chose, tmp
// 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"""
// filter + not use roll up dimension
def mv2_0 = "select o_orderdate, o_shippriority, o_comment, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, " +
"bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 " +
"from orders " +
"group by " +
"o_orderdate, " +
"o_shippriority, " +
"o_comment "
def query2_0 = "select o_shippriority, o_comment, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*) " +
"from orders " +
"where o_shippriority = 2 " +
"group by " +
"o_shippriority, " +
"o_comment "
// rewrite success but cbo not chose, tmp
// order_qt_query2_0_before "${query2_0}"
// check_rewrite(mv2_0, query2_0, "mv2_0")
// order_qt_query2_0_after "${query2_0}"
// sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0"""
// multi table
// filter inside + left + use roll up dimension
def mv13_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query13_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from (select * from lineitem where l_shipdate = '2023-12-11') t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query13_0_before "${query13_0}"
check_rewrite(mv13_0, query13_0, "mv13_0")
order_qt_query13_0_after "${query13_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv13_0"""
// filter inside + right + use roll up dimension
def mv14_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query14_0 = "select l_partkey, l_suppkey, l_shipdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query14_0_before "${query14_0}"
check_rewrite(mv14_0, query14_0, "mv14_0")
order_qt_query14_0_after "${query14_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv14_0"""
// filter inside + right + left + use roll up dimension
def mv15_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join (select * from orders where o_orderstatus = 'o') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query15_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from (select * from lineitem where l_shipdate = '2023-12-11') t1 " +
"left join (select * from orders where o_orderstatus = 'o') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query15_0_before "${query15_0}"
check_rewrite(mv15_0, query15_0, "mv15_0")
order_qt_query15_0_after "${query15_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv15_0"""
// filter outside + left + use roll up dimension
def mv16_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query16_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where l_shipdate = '2023-12-11' " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query16_0_before "${query16_0}"
check_rewrite(mv16_0, query16_0, "mv16_0")
order_qt_query16_0_after "${query16_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv16_0"""
// filter outside + right + use roll up dimension
def mv17_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query17_0 = "select t1.l_partkey, t1.l_suppkey, l_shipdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where o_orderdate = '2023-12-11' " +
"group by " +
"l_shipdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query17_0_before "${query17_0}"
check_rewrite(mv17_0, query17_0, "mv17_0")
order_qt_query17_0_after "${query17_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv17_0"""
// filter outside + left + right + use roll up dimension
def mv18_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query18_0 = "select t1.l_suppkey, l_shipdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where o_orderdate = '2023-12-11' and l_partkey = 2 " +
"group by " +
"l_shipdate, " +
"l_suppkey"
order_qt_query18_0_before "${query18_0}"
check_rewrite(mv18_0, query18_0, "mv18_0")
order_qt_query18_0_after "${query18_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv18_0"""
// filter inside + left + use not roll up dimension
def mv19_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query19_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, sum(o_totalprice) " +
"from (select * from lineitem where l_partkey = 2 ) t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query19_0_before "${query19_0}"
check_rewrite(mv19_0, query19_0, "mv19_0")
order_qt_query19_0_after "${query19_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_0"""
def mv19_1 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from (select * from lineitem where l_partkey = 2) t1 " +
"left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query19_1 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end), " +
"count(*) " +
"from (select * from lineitem where l_partkey = 2 and l_suppkey = 3) t1 " +
"left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
// // Should pass but not, tmp
// order_qt_query19_1_before "${query19_1}"
// check_rewrite(mv19_1, query19_1, "mv19_1")
// order_qt_query19_1_after "${query19_1}"
// sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_1"""
// filter inside + right + use not roll up dimension
def mv20_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join (select * from orders where o_orderstatus = 'o') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query20_0 = "select l_shipdate, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join (select * from orders where o_orderstatus = 'o') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_suppkey"
order_qt_query20_0_before "${query20_0}"
check_rewrite(mv20_0, query20_0, "mv20_0")
order_qt_query20_0_after "${query20_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv20_0"""
// filter inside + right + left + use not roll up dimension
def mv21_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join (select * from orders where o_orderstatus = 'o') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query21_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from (select * from lineitem where l_partkey = 2) t1 " +
"left join (select * from orders where o_orderstatus = 'o') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query21_0_before "${query21_0}"
check_rewrite(mv21_0, query21_0, "mv21_0")
order_qt_query21_0_after "${query21_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv21_0"""
// filter outside + left + use not roll up dimension
def mv22_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query22_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where l_partkey = 2 or l_suppkey = 3 " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query22_0_before "${query22_0}"
check_rewrite(mv22_0, query22_0, "mv22_0")
order_qt_query22_0_after "${query22_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv22_0"""
def mv22_1 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"where l_partkey = 2 " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query22_1 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where l_partkey = 2 and l_suppkey = 3 " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
// Should pass but not, tmp
// order_qt_query22_1_before "${query22_1}"
// check_rewrite(mv22_1, query22_1, "mv22_1")
// order_qt_query22_1_after "${query22_1}"
// sql """ DROP MATERIALIZED VIEW IF EXISTS mv22_0"""
// filter outside + right + use not roll up dimension
def mv23_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderstatus, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey, " +
"o_orderstatus"
def query23_0 = "select t1.l_partkey, t1.l_suppkey, l_shipdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where o_orderdate = '2023-12-08' and o_orderstatus = 'o' " +
"group by " +
"l_shipdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query23_0_before "${query23_0}"
check_rewrite(mv23_0, query23_0, "mv23_0")
order_qt_query23_0_after "${query23_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv23_0"""
// filter outside + left + right + not use roll up dimension
def mv24_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query24_0 = "select t1.l_suppkey, l_shipdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where l_suppkey = 3 " +
"group by " +
"l_shipdate, " +
"l_suppkey"
order_qt_query24_0_before "${query24_0}"
check_rewrite(mv24_0, query24_0, "mv24_0")
order_qt_query24_0_after "${query24_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv24_0"""
// without filter
def mv25_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all " +
"from lineitem " +
"left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query25_0 = "select l_partkey, l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*) " +
"from lineitem " +
"left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query25_0_before "${query25_0}"
check_rewrite(mv25_0, query25_0, "mv25_0")
order_qt_query25_0_after "${query25_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv25_0"""
// can not rewrite, todo
}

View File

@ -22,6 +22,8 @@ suite("aggregate_without_roll_up") {
sql "SET enable_fallback_to_original_planner=false"
sql "SET enable_materialized_view_rewrite=true"
sql "SET enable_nereids_timeout = false"
// tmp disable to rewrite, will be removed in the future
sql "SET disable_nereids_rules = 'INFER_PREDICATES, ELIMINATE_OUTER_JOIN'"
sql """
drop table if exists orders
@ -29,18 +31,19 @@ suite("aggregate_without_roll_up") {
sql """
CREATE TABLE IF NOT EXISTS orders (
O_ORDERKEY INTEGER NOT NULL,
O_CUSTKEY INTEGER NOT NULL,
O_ORDERSTATUS CHAR(1) NOT NULL,
O_TOTALPRICE DECIMALV3(15,2) NOT NULL,
O_ORDERDATE DATE NOT NULL,
O_ORDERPRIORITY CHAR(15) NOT NULL,
O_CLERK CHAR(15) NOT NULL,
O_SHIPPRIORITY INTEGER NOT NULL,
O_COMMENT VARCHAR(79) NOT NULL
o_orderkey INTEGER NOT NULL,
o_custkey INTEGER NOT NULL,
o_orderstatus CHAR(1) NOT NULL,
o_totalprice DECIMALV3(15,2) NOT NULL,
o_orderdate DATE NOT NULL,
o_orderpriority CHAR(15) NOT NULL,
o_clerk CHAR(15) NOT NULL,
o_shippriority INTEGER NOT NULL,
o_comment VARCHAR(79) NOT NULL
)
DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY)
DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3
DUPLICATE KEY(o_orderkey, o_custkey)
PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30'))
DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
@ -52,25 +55,26 @@ suite("aggregate_without_roll_up") {
sql"""
CREATE TABLE IF NOT EXISTS lineitem (
L_ORDERKEY INTEGER NOT NULL,
L_PARTKEY INTEGER NOT NULL,
L_SUPPKEY INTEGER NOT NULL,
L_LINENUMBER INTEGER NOT NULL,
L_QUANTITY DECIMALV3(15,2) NOT NULL,
L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL,
L_DISCOUNT DECIMALV3(15,2) NOT NULL,
L_TAX DECIMALV3(15,2) NOT NULL,
L_RETURNFLAG CHAR(1) NOT NULL,
L_LINESTATUS CHAR(1) NOT NULL,
L_SHIPDATE DATE NOT NULL,
L_COMMITDATE DATE NOT NULL,
L_RECEIPTDATE DATE NOT NULL,
L_SHIPINSTRUCT CHAR(25) NOT NULL,
L_SHIPMODE CHAR(10) NOT NULL,
L_COMMENT VARCHAR(44) NOT NULL
l_orderkey INTEGER NOT NULL,
l_partkey INTEGER NOT NULL,
l_suppkey INTEGER NOT NULL,
l_linenumber INTEGER NOT NULL,
l_quantity DECIMALV3(15,2) NOT NULL,
l_extendedprice DECIMALV3(15,2) NOT NULL,
l_discount DECIMALV3(15,2) NOT NULL,
l_tax DECIMALV3(15,2) NOT NULL,
l_returnflag CHAR(1) NOT NULL,
l_linestatus CHAR(1) NOT NULL,
l_shipdate DATE NOT NULL,
l_commitdate DATE NOT NULL,
l_receiptdate DATE NOT NULL,
l_shipinstruct CHAR(25) NOT NULL,
l_shipmode CHAR(10) NOT NULL,
l_comment VARCHAR(44) NOT NULL
)
DUPLICATE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER)
DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3
DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber)
PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30'))
DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
@ -82,33 +86,39 @@ suite("aggregate_without_roll_up") {
sql """
CREATE TABLE IF NOT EXISTS partsupp (
PS_PARTKEY INTEGER NOT NULL,
PS_SUPPKEY INTEGER NOT NULL,
PS_AVAILQTY INTEGER NOT NULL,
PS_SUPPLYCOST DECIMALV3(15,2) NOT NULL,
PS_COMMENT VARCHAR(199) NOT NULL
ps_partkey INTEGER NOT NULL,
ps_suppkey INTEGER NOT NULL,
ps_availqty INTEGER NOT NULL,
ps_supplycost DECIMALV3(15,2) NOT NULL,
ps_comment VARCHAR(199) NOT NULL
)
DUPLICATE KEY(PS_PARTKEY, PS_SUPPKEY)
DISTRIBUTED BY HASH(PS_PARTKEY) BUCKETS 3
DUPLICATE KEY(ps_partkey, ps_suppkey)
DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
"""
waiteCreateTableFinished("lineitem")
sql """ insert into lineitem values
(1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(2, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-11', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');"""
waiteCreateTableFinished("orders")
sql """
insert into orders values
(1, 1, 'ok', 99.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(2, 2, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm'),
(2, 3, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm');
(2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');
"""
sql """
insert into orders values
(1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'),
(3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'),
(3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'),
(4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'),
(5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'),
(5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi');
"""
waiteCreateTableFinished("partsupp")
sql """
insert into partsupp values
(2, 3, 9, 10.01, 'supply1'),
@ -134,19 +144,625 @@ suite("aggregate_without_roll_up") {
}
}
// select + from + inner join + group by
def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, sum(O_TOTALPRICE) as sum_alias " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"group by lineitem.L_LINENUMBER, orders.O_CUSTKEY "
def query1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, sum(O_TOTALPRICE) as sum_alias " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"group by lineitem.L_LINENUMBER, orders.O_CUSTKEY "
// fix later
// order_qt_query1_0_before "${query1_0}"
check_rewrite(mv1_0, query1_0, "mv1_0")
// order_qt_query1_0_after "${query1_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0"""
}
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})"
}
}
// single table
// with filter
def mv1_0 = "select o_shippriority, o_comment, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2 " +
"from orders " +
"group by " +
"o_shippriority, " +
"o_comment "
def query1_0 = "select o_shippriority, o_comment, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*) " +
"from orders " +
"where o_shippriority in (9.5, 10.5)" +
"group by " +
"o_shippriority, " +
"o_comment "
order_qt_query1_0_before "${query1_0}"
check_rewrite(mv1_0, query1_0, "mv1_0")
order_qt_query1_0_after "${query1_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0"""
def mv1_1 = "select O_SHIPPRIORITY, O_COMMENT, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from orders " +
"where O_ORDERDATE < '2023-12-30'" +
"group by " +
"O_SHIPPRIORITY, " +
"O_COMMENT "
def query1_1 = "select O_SHIPPRIORITY, O_COMMENT, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from orders " +
"where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'" +
"group by " +
"O_SHIPPRIORITY, " +
"O_COMMENT "
// should support but not, tmp
// 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 O_SHIPPRIORITY, O_COMMENT, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from orders " +
"where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'" +
"group by " +
"O_SHIPPRIORITY, " +
"O_COMMENT "
def query1_2 = "select O_SHIPPRIORITY, O_COMMENT, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from orders " +
"where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'" +
"group by " +
"O_SHIPPRIORITY, " +
"O_COMMENT "
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"""
// without filter
def mv2_0 = "select O_SHIPPRIORITY, O_COMMENT, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from orders " +
"group by " +
"O_SHIPPRIORITY, " +
"O_COMMENT "
def query2_0 = "select O_SHIPPRIORITY, O_COMMENT, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from orders " +
"group by " +
"O_SHIPPRIORITY, " +
"O_COMMENT "
order_qt_query2_0_before "${query2_0}"
check_rewrite(mv2_0, query2_0, "mv2_0")
order_qt_query2_0_after "${query2_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0"""
// without group, scalar aggregate
def mv3_0 = "select count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from orders " +
"where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'"
def query3_0 = "select count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from orders " +
"where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'"
order_qt_query3_0_before "${query3_0}"
check_rewrite(mv3_0, query3_0, "mv3_0")
order_qt_query3_0_after "${query3_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0"""
// multi table
// filter inside + left
def mv13_0 = "select o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query13_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from (select * from lineitem where l_partkey = 2) t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query13_0_before "${query13_0}"
check_rewrite(mv13_0, query13_0, "mv13_0")
order_qt_query13_0_after "${query13_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv13_0"""
// filter inside + right
def mv14_0 = "select o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " +
"from lineitem " +
"left join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query14_0 = "select l_partkey, l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query14_0_before "${query14_0}"
check_rewrite(mv14_0, query14_0, "mv14_0")
order_qt_query14_0_after "${query14_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv14_0"""
// filter inside + right + left
def mv15_0 = "select o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " +
"from lineitem " +
"left join (select * from orders where o_orderstatus = 'o') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query15_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from (select * from lineitem where l_partkey in (2, 3)) t1 " +
"left join (select * from orders where o_orderstatus = 'o') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query15_0_before "${query15_0}"
check_rewrite(mv15_0, query15_0, "mv15_0")
order_qt_query15_0_after "${query15_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv15_0"""
// filter outside + left
def mv16_0 = "select o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query16_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where l_partkey in (1, 2 ,3, 4) " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query16_0_before "${query16_0}"
check_rewrite(mv16_0, query16_0, "mv16_0")
order_qt_query16_0_after "${query16_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv16_0"""
// filter outside + right
def mv17_0 = "select o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query17_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where o_orderdate = '2023-12-11' " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query17_0_before "${query17_0}"
check_rewrite(mv17_0, query17_0, "mv17_0")
order_qt_query17_0_after "${query17_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv17_0"""
def mv17_1 = "select L_ORDERKEY, O_SHIPPRIORITY, O_COMMENT, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from lineitem " +
"left join " +
"orders " +
"on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERDATE < '2023-12-30' and orders.O_ORDERDATE > '2023-12-01' " +
"group by " +
"lineitem.L_ORDERKEY, " +
"orders.O_SHIPPRIORITY, " +
"orders.O_COMMENT "
def query17_1 = "select L_ORDERKEY, O_SHIPPRIORITY, O_COMMENT, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17 " +
"from lineitem " +
"left join " +
"orders " +
"on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERDATE < '2023-12-30' and orders.O_ORDERDATE > '2023-12-01' " +
"group by " +
"lineitem.L_ORDERKEY, " +
"orders.O_SHIPPRIORITY, " +
"orders.O_COMMENT "
// rewrite success but cbo not chose, tmp
// order_qt_query17_1_before "${query17_1}"
// check_rewrite(mv17_1, query17_1, "mv17_1")
// order_qt_query17_1_after "${query17_1}"
// sql """ DROP MATERIALIZED VIEW IF EXISTS mv17_1"""
// filter outside + left + right
def mv18_0 = "select l_shipdate, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all, " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"l_shipdate, " +
"l_suppkey"
def query18_0 = "select t1.l_suppkey, l_shipdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*), " +
"count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " +
"from lineitem t1 " +
"left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " +
"where l_shipdate = '2023-12-11' and l_suppkey = 2 " +
"group by " +
"l_shipdate, " +
"l_suppkey"
order_qt_query18_0_before "${query18_0}"
check_rewrite(mv18_0, query18_0, "mv18_0")
order_qt_query18_0_after "${query18_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv18_0"""
def mv18_1 = "select l_linenumber, o_custkey, sum(o_totalprice) as sum_alias " +
"from lineitem " +
"inner join orders on l_orderkey = o_orderkey " +
"group by l_linenumber, o_custkey "
def query18_1 = "select l_linenumber, sum(o_totalprice) as sum_alias " +
"from lineitem " +
"inner join orders on l_orderkey = o_orderkey " +
"where o_custkey = 2 and l_linenumber = 3 " +
"group by l_linenumber, o_custkey "
order_qt_query18_1_before "${query18_1}"
check_rewrite(mv18_1, query18_1, "mv18_1")
order_qt_query18_1_after "${query18_1}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv18_1"""
def mv18_2 = "select lineitem.l_linenumber, orders.o_custkey, sum(o_totalprice) as sum_alias " +
"from lineitem " +
"inner join orders on l_orderkey = o_orderkey " +
"group by lineitem.l_linenumber, orders.o_custkey "
def query18_2 = "select lineitem.l_linenumber, sum(o_totalprice) as sum_alias " +
"from lineitem " +
"inner join orders on lineitem.l_orderkey = orders.o_orderkey " +
"where o_custkey = 2 and l_suppkey= 4 " +
"group by lineitem.l_linenumber, orders.o_custkey "
order_qt_query18_2_before "${query18_2}"
check_not_match(mv18_2, query18_2, "mv18_2")
order_qt_query18_2_after "${query18_2}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv18_2"""
// without filter
def mv19_0 = "select o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
"max(o_totalprice) as max_total, " +
"min(o_totalprice) as min_total, " +
"count(*) as count_all " +
"from lineitem " +
"left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
def query19_0 = "select l_partkey, l_suppkey, o_orderdate, " +
"sum(o_totalprice), " +
"max(o_totalprice), " +
"min(o_totalprice), " +
"count(*) " +
"from lineitem " +
"left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " +
"group by " +
"o_orderdate, " +
"l_partkey, " +
"l_suppkey"
order_qt_query19_0_before "${query19_0}"
check_rewrite(mv19_0, query19_0, "mv19_0")
order_qt_query19_0_after "${query19_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_0"""
def mv19_1 = "select lineitem.l_linenumber, orders.o_custkey, sum(o_totalprice) as sum_alias " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"group by lineitem.L_LINENUMBER, orders.O_CUSTKEY "
def query19_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, sum(O_TOTALPRICE) as sum_alias " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"group by lineitem.L_LINENUMBER, orders.O_CUSTKEY "
order_qt_query19_1_before "${query19_1}"
check_rewrite(mv19_1, query19_1, "mv19_1")
order_qt_query19_1_after "${query19_1}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_1"""
// without group, scalar aggregate
def mv20_0 = "select count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " +
"from lineitem " +
"left join " +
"orders " +
"on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERDATE < '2023-12-30' and orders.O_ORDERDATE > '2023-12-01' "
def query20_0 = "select count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " +
"count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " +
"count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " +
"count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17 " +
"from lineitem " +
"left join " +
"orders " +
"on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERDATE < '2023-12-30' and orders.O_ORDERDATE > '2023-12-01' "
// rewrite success but cbo not chose, tmp
// order_qt_query20_0_before "${query20_0}"
// check_rewrite(mv20_0, query20_0, "mv20_0")
// order_qt_query20_0_after "${query20_0}"
// sql """ DROP MATERIALIZED VIEW IF EXISTS mv20_0"""
}

View File

@ -0,0 +1,428 @@
// 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("inner_join") {
String db = context.config.getDbNameByFile(context.file)
sql "use ${db}"
sql "SET enable_nereids_planner=true"
sql "SET enable_fallback_to_original_planner=false"
sql "SET enable_materialized_view_rewrite=true"
sql "SET enable_nereids_timeout = false"
// tmp disable to rewrite, will be removed in the future
sql "SET disable_nereids_rules = 'INFER_PREDICATES, ELIMINATE_OUTER_JOIN'"
sql """
drop table if exists orders
"""
sql """
CREATE TABLE IF NOT EXISTS orders (
o_orderkey INTEGER NOT NULL,
o_custkey INTEGER NOT NULL,
o_orderstatus CHAR(1) NOT NULL,
o_totalprice DECIMALV3(15,2) NOT NULL,
o_orderdate DATE NOT NULL,
o_orderpriority CHAR(15) NOT NULL,
o_clerk CHAR(15) NOT NULL,
o_shippriority INTEGER NOT NULL,
o_comment VARCHAR(79) NOT NULL
)
DUPLICATE KEY(o_orderkey, o_custkey)
PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30'))
DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
"""
sql """
drop table if exists lineitem
"""
sql"""
CREATE TABLE IF NOT EXISTS lineitem (
l_orderkey INTEGER NOT NULL,
l_partkey INTEGER NOT NULL,
l_suppkey INTEGER NOT NULL,
l_linenumber INTEGER NOT NULL,
l_quantity DECIMALV3(15,2) NOT NULL,
l_extendedprice DECIMALV3(15,2) NOT NULL,
l_discount DECIMALV3(15,2) NOT NULL,
l_tax DECIMALV3(15,2) NOT NULL,
l_returnflag CHAR(1) NOT NULL,
l_linestatus CHAR(1) NOT NULL,
l_shipdate DATE NOT NULL,
l_commitdate DATE NOT NULL,
l_receiptdate DATE NOT NULL,
l_shipinstruct CHAR(25) NOT NULL,
l_shipmode CHAR(10) NOT NULL,
l_comment VARCHAR(44) NOT NULL
)
DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber)
PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30'))
DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
"""
sql """
drop table if exists partsupp
"""
sql """
CREATE TABLE IF NOT EXISTS partsupp (
ps_partkey INTEGER NOT NULL,
ps_suppkey INTEGER NOT NULL,
ps_availqty INTEGER NOT NULL,
ps_supplycost DECIMALV3(15,2) NOT NULL,
ps_comment VARCHAR(199) NOT NULL
)
DUPLICATE KEY(ps_partkey, ps_suppkey)
DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
"""
sql """ insert into lineitem values
(1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');
"""
sql """
insert into orders values
(1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'),
(3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'),
(3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'),
(4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'),
(5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'),
(5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi');
"""
sql """
insert into partsupp values
(2, 3, 9, 10.01, 'supply1'),
(2, 3, 10, 11.01, 'supply2');
"""
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})"
}
}
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})"
}
}
// without filter
def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_0 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
order_qt_query1_0_before "${query1_0}"
check_rewrite(mv1_0, query1_0, "mv1_0")
order_qt_query1_0_after "${query1_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0"""
def mv1_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
def query1_1 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
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 lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from orders " +
"inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_2 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
order_qt_query1_2_before "${query1_2}"
// join direction is not same, should not match
check_rewrite(mv1_2, query1_2, "mv1_2")
order_qt_query1_2_after "${query1_2}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_2"""
// select with complex expression
def mv1_3 = "select l_linenumber, o_custkey " +
"from orders " +
"inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_3 = "select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null, " +
"case when l_linenumber in (1,2,3) then l_linenumber else o_custkey end as case_when " +
"from orders " +
"inner join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY"
order_qt_query1_3_before "${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 mv1_4 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
def query1_4 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
order_qt_query1_4_before "${query1_4}"
check_rewrite(mv1_4, query1_4, "mv1_4")
order_qt_query1_4_after "${query1_4}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_4"""
// filter outside + left
def mv2_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from orders " +
"inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query2_0 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where lineitem.L_LINENUMBER > 10"
order_qt_query2_0_before "${query2_0}"
check_rewrite(mv2_0, query2_0, "mv2_0")
order_qt_query2_0_after "${query2_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0"""
def mv2_1 = "select t1.L_LINENUMBER, orders.O_CUSTKEY " +
"from (select * from lineitem where L_LINENUMBER > 1) t1 " +
"inner join orders on t1.L_ORDERKEY = orders.O_ORDERKEY "
def query2_1 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where lineitem.L_LINENUMBER > 1"
order_qt_query2_1_before "${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 t1.L_LINENUMBER, orders.O_CUSTKEY " +
"from (select * from lineitem where L_LINENUMBER > 1) t1 " +
"inner join orders on t1.L_ORDERKEY = orders.O_ORDERKEY "
def query2_2 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where lineitem.L_LINENUMBER > 1 and l_suppkey = 3"
// Should success but not, because mv contains the part filter of mv, tmp
// order_qt_query2_2_before "${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 lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY, l_suppkey " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
def query2_3= "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY " +
"where lineitem.L_LINENUMBER > 1 and l_suppkey = 3 "
order_qt_query2_3_before "${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"""
// filter outside + right
def mv3_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query3_0 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERSTATUS = 'o'"
order_qt_query3_0_before "${query3_0}"
// use a filed not from mv, should not success
check_not_match(mv3_0, query3_0, "mv3_0")
order_qt_query3_0_after "${query3_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0"""
def mv3_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, orders.O_ORDERSTATUS " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query3_1 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERSTATUS = 'o'"
order_qt_query3_1_before "${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 lineitem.L_LINENUMBER, t2.O_CUSTKEY, t2.O_ORDERSTATUS " +
"from lineitem " +
"inner join " +
"(select * from orders where O_ORDERSTATUS = 'o') t2 " +
"on lineitem.L_ORDERKEY = t2.O_ORDERKEY "
def query3_2 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERSTATUS = 'o'"
order_qt_query3_2_before "${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 lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
def query3_3= "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY " +
"where o_custkey in (1, 2, 3, 4) "
order_qt_query3_3_before "${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"""
// filter outside + left + right
def mv4_0 = "select l_linenumber, o_custkey, o_orderkey, o_orderstatus " +
"from lineitem " +
"inner join orders on lineitem.l_orderkey = orders.o_orderkey "
def query4_0 = "select lineitem.l_linenumber " +
"from lineitem " +
"inner join orders on lineitem.l_orderkey = orders.o_orderkey " +
"where o_orderstatus = 'o' AND l_linenumber in (1, 2, 3, 4, 5) "
order_qt_query4_0_before "${query4_0}"
check_rewrite(mv4_0, query4_0, "mv4_0")
order_qt_query4_0_after "${query4_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_0"""
// filter inside + left
def mv5_0 = "select lineitem.l_linenumber, orders.o_custkey " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where lineitem.L_LINENUMBER > 1"
def query5_0 = "select t1.L_LINENUMBER " +
"from (select * from lineitem where l_linenumber > 1) t1 " +
"inner join orders on t1.l_orderkey = orders.O_ORDERKEY "
order_qt_query5_0_before "${query5_0}"
check_rewrite(mv5_0, query5_0, "mv5_0")
order_qt_query5_0_after "${query5_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv5_0"""
// filter inside + right
def mv6_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " +
"from lineitem " +
"inner join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate "
def query6_0 = "select l_partkey, l_suppkey, l_shipdate " +
"from lineitem t1 " +
"inner join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate "
// should passed but not as isGraphLogicalEquals is false
// order_qt_query6_0_before "${query6_0}"
// check_rewrite(mv6_0, query6_0, "mv6_0")
// order_qt_query6_0_after "${query6_0}"
// sql """ DROP MATERIALIZED VIEW IF EXISTS mv6_0"""
// filter inside + inner + right
def mv7_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " +
"from lineitem " +
"inner join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate "
def query7_0 = "select l_partkey, l_suppkey, l_shipdate " +
"from (select l_shipdate, l_orderkey, l_partkey, l_suppkey " +
"from lineitem where l_partkey in (3, 4)) t1 " +
"inner join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " +
"where l_partkey = 3"
// should passed but not, because isGraphLogicalEquals is false
// order_qt_query7_0_before "${query7_0}"
// check_rewrite(mv7_0, query7_0, "mv7_0")
// order_qt_query7_0_after "${query7_0}"
// sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_0"""
// check not match, because use a filed orders.O_SHIPPRIORITY which not in mv
def mv10_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY"
def query10_0 = "select orders.O_CUSTKEY " +
"from orders " +
"inner join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY " +
"WHERE lineitem.L_LINENUMBER > 10 AND orders.O_CUSTKEY = 5 AND " +
"orders.O_SHIPPRIORITY = 1"
order_qt_query10_0_before "${query10_0}"
check_not_match(mv10_0, query10_0, "mv10_0")
order_qt_query10_0_after "${query10_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv10_0"""
}

View File

@ -1,207 +0,0 @@
// 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("inner_join") {
String db = context.config.getDbNameByFile(context.file)
sql "use ${db}"
sql "SET enable_nereids_planner=true"
sql "SET enable_fallback_to_original_planner=false"
sql "SET enable_materialized_view_rewrite=true"
sql "SET enable_nereids_timeout = false"
sql """
drop table if exists orders
"""
sql """
CREATE TABLE IF NOT EXISTS orders (
O_ORDERKEY INTEGER NOT NULL,
O_CUSTKEY INTEGER NOT NULL,
O_ORDERSTATUS CHAR(1) NOT NULL,
O_TOTALPRICE DECIMALV3(15,2) NOT NULL,
O_ORDERDATE DATE NOT NULL,
O_ORDERPRIORITY CHAR(15) NOT NULL,
O_CLERK CHAR(15) NOT NULL,
O_SHIPPRIORITY INTEGER NOT NULL,
O_COMMENT VARCHAR(79) NOT NULL
)
DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY)
DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
"""
sql """
drop table if exists lineitem
"""
sql"""
CREATE TABLE IF NOT EXISTS lineitem (
L_ORDERKEY INTEGER NOT NULL,
L_PARTKEY INTEGER NOT NULL,
L_SUPPKEY INTEGER NOT NULL,
L_LINENUMBER INTEGER NOT NULL,
L_QUANTITY DECIMALV3(15,2) NOT NULL,
L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL,
L_DISCOUNT DECIMALV3(15,2) NOT NULL,
L_TAX DECIMALV3(15,2) NOT NULL,
L_RETURNFLAG CHAR(1) NOT NULL,
L_LINESTATUS CHAR(1) NOT NULL,
L_SHIPDATE DATE NOT NULL,
L_COMMITDATE DATE NOT NULL,
L_RECEIPTDATE DATE NOT NULL,
L_SHIPINSTRUCT CHAR(25) NOT NULL,
L_SHIPMODE CHAR(10) NOT NULL,
L_COMMENT VARCHAR(44) NOT NULL
)
DUPLICATE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER)
DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
"""
sql """
drop table if exists partsupp
"""
sql """
CREATE TABLE IF NOT EXISTS partsupp (
PS_PARTKEY INTEGER NOT NULL,
PS_SUPPKEY INTEGER NOT NULL,
PS_AVAILQTY INTEGER NOT NULL,
PS_SUPPLYCOST DECIMALV3(15,2) NOT NULL,
PS_COMMENT VARCHAR(199) NOT NULL
)
DUPLICATE KEY(PS_PARTKEY, PS_SUPPKEY)
DISTRIBUTED BY HASH(PS_PARTKEY) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
)
"""
waiteCreateTableFinished("lineitem")
sql """ insert into lineitem values
(1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(2, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-11', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');"""
waiteCreateTableFinished("orders")
sql """
insert into orders values
(1, 1, 'ok', 99.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(2, 2, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm'),
(2, 3, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm');
"""
waiteCreateTableFinished("partsupp")
sql """
insert into partsupp values
(2, 3, 9, 10.01, 'supply1'),
(2, 3, 10, 11.01, 'supply2');
"""
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})"
}
}
// // select + from + inner join
def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_0 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
// fix later
// order_qt_query1_0_before "${query1_0}"
check_rewrite(mv1_0, query1_0, "mv1_0")
// order_qt_query1_0_after "${query1_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0"""
def mv1_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
def query1_1 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
// fix later
// 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 lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from orders " +
"inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_2 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
// fix later
// 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"""
// select + from + inner join + filter
def mv1_3 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from orders " +
"inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_3 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where lineitem.L_LINENUMBER > 10"
// fix later
// order_qt_query1_3_before "${query1_3}"
check_rewrite(mv1_3, query1_3, "mv1_3")
// tmp annotation, will fix later
// order_qt_query1_3_after "${query1_3}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_3"""
// select with complex expression + from + inner join
def mv1_4 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from orders " +
"inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_4 = "select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null " +
"from orders " +
"inner join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY"
// fix later
// order_qt_query1_4_before "${query1_4}"
check_rewrite(mv1_4, query1_4, "mv1_4")
// order_qt_query1_4_after "${query1_4}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_4"""
}

View File

@ -0,0 +1,364 @@
// 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("outer_join") {
String db = context.config.getDbNameByFile(context.file)
sql "use ${db}"
sql "SET enable_nereids_planner=true"
sql "SET enable_fallback_to_original_planner=false"
sql "SET enable_materialized_view_rewrite=true"
sql "SET enable_nereids_timeout = false"
// tmp disable to rewrite, will be removed in the future
sql "SET disable_nereids_rules = 'INFER_PREDICATES, ELIMINATE_OUTER_JOIN'"
sql """
drop table if exists orders
"""
sql """
CREATE TABLE IF NOT EXISTS orders (
o_orderkey INTEGER NOT NULL,
o_custkey INTEGER NOT NULL,
o_orderstatus CHAR(1) NOT NULL,
o_totalprice DECIMALV3(15,2) NOT NULL,
o_orderdate DATE NOT NULL,
o_orderpriority CHAR(15) NOT NULL,
o_clerk CHAR(15) NOT NULL,
o_shippriority INTEGER NOT NULL,
O_COMMENT VARCHAR(79) NOT NULL
)
DUPLICATE KEY(o_orderkey, o_custkey)
PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30'))
DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
);
"""
sql """
drop table if exists lineitem
"""
sql"""
CREATE TABLE IF NOT EXISTS lineitem (
l_orderkey INTEGER NOT NULL,
l_partkey INTEGER NOT NULL,
l_suppkey INTEGER NOT NULL,
l_linenumber INTEGER NOT NULL,
l_quantity DECIMALV3(15,2) NOT NULL,
l_extendedprice DECIMALV3(15,2) NOT NULL,
l_discount DECIMALV3(15,2) NOT NULL,
l_tax DECIMALV3(15,2) NOT NULL,
l_returnflag CHAR(1) NOT NULL,
l_linestatus CHAR(1) NOT NULL,
l_shipdate DATE NOT NULL,
l_commitdate DATE NOT NULL,
l_receiptdate DATE NOT NULL,
l_shipinstruct CHAR(25) NOT NULL,
l_shipmode CHAR(10) NOT NULL,
l_comment VARCHAR(44) NOT NULL
)
DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber)
PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30'))
DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
);
"""
sql """
drop table if exists partsupp
"""
sql """
CREATE TABLE IF NOT EXISTS partsupp (
ps_partkey INTEGER NOT NULL,
ps_suppkey INTEGER NOT NULL,
ps_availqty INTEGER NOT NULL,
ps_supplycost DECIMALV3(15,2) NOT NULL,
ps_comment VARCHAR(199) NOT NULL
)
DUPLICATE KEY(ps_partkey, ps_suppkey)
DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
);
"""
sql """ insert into lineitem values
(1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'),
(5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');
"""
sql """
insert into orders values
(1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'),
(2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'),
(3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'),
(3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'),
(4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'),
(5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'),
(5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi');
"""
sql """
insert into partsupp values
(2, 3, 9, 10.01, 'supply1'),
(2, 3, 10, 11.01, 'supply2');
"""
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})"
}
}
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})"
}
}
// without filter
def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_0 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
order_qt_query1_0_before "${query1_0}"
check_rewrite(mv1_0, query1_0, "mv1_0")
order_qt_query1_0_after "${query1_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0"""
def mv1_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"left join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
def query1_1 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"left join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " +
"and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY"
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 lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from orders " +
"left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_2 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
order_qt_query1_2_before "${query1_2}"
// join direction is not same, should not match
check_not_match(mv1_2, query1_2, "mv1_2")
order_qt_query1_2_after "${query1_2}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_2"""
// select with complex expression
def mv1_3 = "select l_linenumber, o_custkey " +
"from orders " +
"left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query1_3 = "select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null, " +
"case when l_linenumber in (1,2,3) then l_linenumber else o_custkey end as case_when " +
"from orders " +
"left join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY"
order_qt_query1_3_before "${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"""
// filter outside + left
def mv2_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from orders " +
"left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query2_0 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where lineitem.L_LINENUMBER > 10"
order_qt_query2_0_before "${query2_0}"
check_not_match(mv2_0, query2_0, "mv2_0")
order_qt_query2_0_after "${query2_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0"""
def mv2_1 = "select t1.L_LINENUMBER, orders.O_CUSTKEY " +
"from (select * from lineitem where L_LINENUMBER > 1) t1 " +
"left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY "
def query2_1 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where lineitem.L_LINENUMBER > 1"
order_qt_query2_1_before "${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 t1.L_LINENUMBER, orders.O_CUSTKEY " +
"from (select * from lineitem where L_LINENUMBER > 1) t1 " +
"left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY "
def query2_2 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where lineitem.L_LINENUMBER > 1 and l_suppkey = 3"
// Should success but not, tmp
// order_qt_query2_2_before "${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"""
// filter outside + right
def mv3_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query3_0 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERSTATUS = 'o'"
order_qt_query3_0_before "${query3_0}"
// use a filed not from mv, should not success
check_not_match(mv3_0, query3_0, "mv3_0")
order_qt_query3_0_after "${query3_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0"""
def mv3_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, orders.O_ORDERSTATUS " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY "
def query3_1 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERSTATUS = 'o'"
order_qt_query3_1_before "${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 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 "
def query3_2 = "select lineitem.L_LINENUMBER " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where orders.O_ORDERSTATUS = 'o'"
order_qt_query3_2_before "${query3_2}"
// should not success, as mv filter is under left outer input
check_not_match(mv3_2, query3_2, "mv3_2")
order_qt_query3_2_after "${query3_2}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_2"""
// filter outside + left + right
def mv4_0 = "select l_linenumber, o_custkey, o_orderkey, o_orderstatus " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey "
def query4_0 = "select lineitem.l_linenumber " +
"from lineitem " +
"left join orders on lineitem.l_orderkey = orders.o_orderkey " +
"where o_orderstatus = 'o' AND o_orderkey = 1"
order_qt_query4_0_before "${query4_0}"
check_rewrite(mv4_0, query4_0, "mv4_0")
order_qt_query4_0_after "${query4_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_0"""
// filter inside + left
def mv5_0 = "select lineitem.l_linenumber, orders.o_custkey " +
"from lineitem " +
"left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " +
"where lineitem.L_LINENUMBER > 1"
def query5_0 = "select t1.L_LINENUMBER " +
"from (select * from lineitem where l_linenumber > 1) t1 " +
"left join orders on t1.l_orderkey = orders.O_ORDERKEY "
order_qt_query5_0_before "${query5_0}"
check_rewrite(mv5_0, query5_0, "mv5_0")
order_qt_query5_0_after "${query5_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv5_0"""
// filter inside + right
def mv6_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " +
"from lineitem " +
"left join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate "
def query6_0 = "select l_partkey, l_suppkey, l_shipdate " +
"from lineitem t1 " +
"left join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate "
order_qt_query6_0_before "${query6_0}"
check_rewrite(mv6_0, query6_0, "mv6_0")
order_qt_query6_0_after "${query6_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv6_0"""
// filter inside + left + right
def mv7_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " +
"from lineitem " +
"left join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate "
def query7_0 = "select l_partkey, l_suppkey, l_shipdate " +
"from (select l_shipdate, l_orderkey, l_partkey, l_suppkey " +
"from lineitem where l_partkey in (3, 4)) t1 " +
"left join (select * from orders where o_orderdate = '2023-12-08') t2 " +
"on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " +
"where l_partkey = 3"
order_qt_query7_0_before "${query7_0}"
check_rewrite(mv7_0, query7_0, "mv7_0")
order_qt_query7_0_after "${query7_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_0"""
}

View File

@ -0,0 +1,176 @@
// 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("partition_mv_rewrite") {
String db = context.config.getDbNameByFile(context.file)
sql "use ${db}"
sql "SET enable_nereids_planner=true"
sql "SET enable_fallback_to_original_planner=false"
sql "SET enable_materialized_view_rewrite=true"
sql "SET enable_nereids_timeout = false"
// tmp disable to rewrite, will be removed in the future
sql "SET disable_nereids_rules = 'INFER_PREDICATES, ELIMINATE_OUTER_JOIN'"
sql """
drop table if exists orders
"""
sql """
CREATE TABLE IF NOT EXISTS orders (
o_orderkey integer not null,
o_custkey integer not null,
o_orderstatus char(1) not null,
o_totalprice decimalv3(15,2) not null,
o_orderdate date not null,
o_orderpriority char(15) not null,
o_clerk char(15) not null,
o_shippriority integer not null,
o_comment varchar(79) not null
)
DUPLICATE KEY(o_orderkey, o_custkey)
PARTITION BY RANGE(o_orderdate)(
FROM ('2023-10-17') TO ('2023-10-20') INTERVAL 1 DAY
)
DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
);
"""
sql """
drop table if exists lineitem
"""
sql"""
CREATE TABLE IF NOT EXISTS lineitem (
l_orderkey integer not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimalv3(15,2) not null,
l_extendedprice decimalv3(15,2) not null,
l_discount decimalv3(15,2) not null,
l_tax decimalv3(15,2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null
)
DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber)
PARTITION BY RANGE(l_shipdate)
(FROM ('2023-10-17') TO ('2023-10-20') INTERVAL 1 DAY)
DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3
PROPERTIES (
"replication_num" = "1"
);
"""
sql"""
insert into orders values
(1, 1, 'ok', 99.5, '2023-10-17', 'a', 'b', 1, 'yy'),
(2, 2, 'ok', 109.2, '2023-10-18', 'c','d',2, 'mm'),
(3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy');
"""
sql """
insert into lineitem values
(1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy'),
(2, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy'),
(3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx');
"""
def mv_def_sql = """
select l_shipdate, o_orderdate, l_partkey,
l_suppkey, sum(o_totalprice) as sum_total
from lineitem
left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate
group by
l_shipdate,
o_orderdate,
l_partkey,
l_suppkey;
"""
def all_partition_sql = """
select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total
from lineitem
left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate
group by
l_shipdate,
o_orderdate,
l_partkey,
l_suppkey;
"""
def partition_sql = """
select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total
from lineitem
left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate
where (l_shipdate>= '2023-10-18' and l_shipdate <= '2023-10-19')
group by
l_shipdate,
o_orderdate,
l_partkey,
l_suppkey;
"""
sql """DROP MATERIALIZED VIEW IF EXISTS mv_10086"""
sql """DROP TABLE IF EXISTS mv_10086"""
sql"""
CREATE MATERIALIZED VIEW mv_10086
BUILD IMMEDIATE REFRESH AUTO ON MANUAL
partition by(l_shipdate)
DISTRIBUTED BY RANDOM BUCKETS 2
PROPERTIES ('replication_num' = '1')
AS
${mv_def_sql}
"""
def job_name = getJobName(db, "mv_10086");
waitingMTMVTaskFinished(job_name)
explain {
sql("${all_partition_sql}")
contains "mv_10086"
}
explain {
sql("${partition_sql}")
contains "mv_10086"
}
// partition is invalid, so can not use partition 2023-10-17 to rewrite
sql """
insert into lineitem values
(1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy');
"""
// wait partition is invalid
sleep(5000)
// only can use valid partition
explain {
sql("${all_partition_sql}")
notContains "mv_10086"
}
explain {
sql("${partition_sql}")
contains "mv_10086"
}
}